You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2017/04/24 20:19:38 UTC

[01/13] geode git commit: GEODE-576 & GEODE-516: GemFireDeadlockDetectorDUnitTest failures [Forced Update!]

Repository: geode
Updated Branches:
  refs/heads/feature/GEODE-2632-6-1 a73e2664c -> 915a0c53c (forced update)


GEODE-576 & GEODE-516: GemFireDeadlockDetectorDUnitTest failures

Replaced pauses with Awaitility.  Modified asyncs to use the DUnit
blackboard to synchronize their actions for repeatable behavior.
Cleaned up static locks to allow their reuse in other tests or in
repeating the same test.


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

Branch: refs/heads/feature/GEODE-2632-6-1
Commit: 50686b0b44024d2bcbb4bea8a36ce3a40ac158c2
Parents: 5891ed7
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Apr 21 09:05:07 2017 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Apr 21 09:06:14 2017 -0700

----------------------------------------------------------------------
 .../GemFireDeadlockDetectorDUnitTest.java       | 116 +++++++++++--------
 .../geode/test/dunit/DUnitBlackboard.java       |  13 +++
 .../test/dunit/internal/InternalBlackboard.java |   5 +
 .../dunit/internal/InternalBlackboardImpl.java  |   5 +
 4 files changed, 91 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/50686b0b/geode-core/src/test/java/org/apache/geode/distributed/internal/deadlock/GemFireDeadlockDetectorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/deadlock/GemFireDeadlockDetectorDUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/deadlock/GemFireDeadlockDetectorDUnitTest.java
index e0bbde0..4a03c2d 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/deadlock/GemFireDeadlockDetectorDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/deadlock/GemFireDeadlockDetectorDUnitTest.java
@@ -14,34 +14,14 @@
  */
 package org.apache.geode.distributed.internal.deadlock;
 
-import org.apache.geode.test.dunit.ThreadUtils;
-import org.junit.experimental.categories.Category;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-import org.awaitility.Awaitility;
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.cache30.CacheTestCase;
 import org.apache.geode.distributed.DistributedLockService;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.LockServiceDestroyedException;
@@ -54,7 +34,20 @@ import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.junit.categories.FlakyTest;
+import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.awaitility.Awaitility;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
 @Category(DistributedTest.class)
 public class GemFireDeadlockDetectorDUnitTest extends JUnit4CacheTestCase {
@@ -108,7 +101,8 @@ public class GemFireDeadlockDetectorDUnitTest extends JUnit4CacheTestCase {
 
   private static final Lock lock = new ReentrantLock();
 
-  @Category(FlakyTest.class) // GEODE-516 & GEODE-576: async actions, thread sleeps, time sensitive
+  // @Category(FlakyTest.class) // GEODE-516 & GEODE-576: async actions, thread sleeps, time
+  // sensitive
   @Test
   public void testDistributedDeadlockWithFunction() throws Throwable {
     Host host = Host.getHost(0);
@@ -117,41 +111,62 @@ public class GemFireDeadlockDetectorDUnitTest extends JUnit4CacheTestCase {
     getSystem();
     InternalDistributedMember member1 = createCache(vm0);
     final InternalDistributedMember member2 = createCache(vm1);
+    getBlackboard().initBlackboard();
 
     // Have two threads lock locks on different members in different orders.
 
+    String gateOnMember1 = "gateOnMember1";
+    String gateOnMember2 = "gateOnMember2";
 
     // This thread locks the lock member1 first, then member2.
-    AsyncInvocation async1 = lockTheLocks(vm0, member2);
-    // This thread locks the lock member2 first, then member1.
-    AsyncInvocation async2 = lockTheLocks(vm1, member1);
+    AsyncInvocation async1 = lockTheLocks(vm0, member2, gateOnMember1, gateOnMember2);
 
-    Thread.sleep(5000);
-    GemFireDeadlockDetector detect = new GemFireDeadlockDetector();
-    LinkedList<Dependency> deadlock = detect.find().findCycle();
-    LogWriterUtils.getLogWriter().info("Deadlock=" + DeadlockDetector.prettyFormat(deadlock));
-    assertEquals(8, deadlock.size());
-    stopStuckThreads();
-    async1.getResult(30000);
-    async2.getResult(30000);
+    // This thread locks the lock member2 first, then member1.
+    AsyncInvocation async2 = lockTheLocks(vm1, member1, gateOnMember2, gateOnMember1);
+    try {
+      final LinkedList<Dependency> deadlockHolder[] = new LinkedList[1];
+      Awaitility.await("waiting for deadlock").atMost(20, TimeUnit.SECONDS).until(() -> {
+        GemFireDeadlockDetector detect = new GemFireDeadlockDetector();
+        LinkedList<Dependency> deadlock = detect.find().findCycle();
+        if (deadlock != null) {
+          deadlockHolder[0] = deadlock;
+        }
+        return deadlock != null;
+      });
+      LinkedList<Dependency> deadlock = deadlockHolder[0];
+      LogWriterUtils.getLogWriter().info("Deadlock=" + DeadlockDetector.prettyFormat(deadlock));
+      assertEquals(8, deadlock.size());
+      stopStuckThreads();
+    } finally {
+      try {
+        waitForAsyncInvocation(async1, 45, TimeUnit.SECONDS);
+      } finally {
+        waitForAsyncInvocation(async2, 45, TimeUnit.SECONDS);
+      }
+    }
   }
 
 
 
-  private AsyncInvocation lockTheLocks(VM vm0, final InternalDistributedMember member) {
+  private AsyncInvocation lockTheLocks(VM vm0, final InternalDistributedMember member,
+      final String gateToSignal, final String gateToWaitOn) {
     return vm0.invokeAsync(new SerializableRunnable() {
 
       public void run() {
         lock.lock();
         try {
-          Thread.sleep(1000);
-        } catch (InterruptedException e) {
-          Assert.fail("interrupted", e);
+          try {
+            getBlackboard().signalGate(gateToSignal);
+            getBlackboard().waitForGate(gateToWaitOn, 10, TimeUnit.SECONDS);
+          } catch (TimeoutException | InterruptedException e) {
+            throw new RuntimeException("failed", e);
+          }
+          ResultCollector collector = FunctionService.onMember(member).execute(new TestFunction());
+          // wait the function to lock the lock on member.
+          collector.getResult();
+        } finally {
+          lock.unlock();
         }
-        ResultCollector collector = FunctionService.onMember(member).execute(new TestFunction());
-        // wait the function to lock the lock on member.
-        collector.getResult();
-        lock.unlock();
       }
     });
   }
@@ -244,14 +259,19 @@ public class GemFireDeadlockDetectorDUnitTest extends JUnit4CacheTestCase {
 
 
     public void execute(FunctionContext context) {
+      boolean acquired = false;
       try {
         stuckThreads.add(Thread.currentThread());
-        lock.tryLock(LOCK_WAIT_TIME, TimeUnit.SECONDS);
+        acquired = lock.tryLock(LOCK_WAIT_TIME, TimeUnit.SECONDS);
       } catch (InterruptedException e) {
-        // ingore
+        // ignore
+      } finally {
+        if (acquired) {
+          lock.unlock();
+        }
+        stuckThreads.remove(Thread.currentThread());
+        context.getResultSender().lastResult(null);
       }
-      stuckThreads.remove(Thread.currentThread());
-      context.getResultSender().lastResult(null);
     }
 
     public String getId() {

http://git-wip-us.apache.org/repos/asf/geode/blob/50686b0b/geode-core/src/test/java/org/apache/geode/test/dunit/DUnitBlackboard.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/DUnitBlackboard.java b/geode-core/src/test/java/org/apache/geode/test/dunit/DUnitBlackboard.java
index a097cd4e..62c92bd 100755
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/DUnitBlackboard.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/DUnitBlackboard.java
@@ -56,6 +56,7 @@ public class DUnitBlackboard {
    * signals a boolean gate
    */
   public void signalGate(String gateName) {
+    // System.out.println(Thread.currentThread().getName()+": signaling gate " + gateName);
     try {
       blackboard.signalGate(gateName);
     } catch (RemoteException e) {
@@ -68,6 +69,7 @@ public class DUnitBlackboard {
    */
   public void waitForGate(String gateName, long timeout, TimeUnit units)
       throws TimeoutException, InterruptedException {
+    // System.out.println(Thread.currentThread().getName()+": waiting for gate " + gateName);
     try {
       blackboard.waitForGate(gateName, timeout, units);
     } catch (RemoteException e) {
@@ -77,6 +79,17 @@ public class DUnitBlackboard {
   }
 
   /**
+   * clear a gate
+   */
+  public void clearGate(String gateName) {
+    try {
+      blackboard.clearGate(gateName);
+    } catch (RemoteException e) {
+      throw new RuntimeException("remote call failed", e);
+    }
+  }
+
+  /**
    * test to see if a gate has been signeled
    */
   public boolean isGateSignaled(String gateName) {

http://git-wip-us.apache.org/repos/asf/geode/blob/50686b0b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboard.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboard.java b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboard.java
index 63f833b..bc5b9b7 100755
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboard.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboard.java
@@ -50,6 +50,11 @@ public interface InternalBlackboard extends Remote, Serializable {
       throws RemoteException, TimeoutException, InterruptedException;
 
   /**
+   * clears a gate
+   */
+  void clearGate(String gateName) throws RemoteException;
+
+  /**
    * test to see if a gate has been signeled
    */
   boolean isGateSignaled(String gateName) throws RemoteException;

http://git-wip-us.apache.org/repos/asf/geode/blob/50686b0b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboardImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboardImpl.java b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboardImpl.java
index e7657ed..feeae15 100755
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboardImpl.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/internal/InternalBlackboardImpl.java
@@ -79,6 +79,11 @@ public class InternalBlackboardImpl extends UnicastRemoteObject implements Inter
   }
 
   @Override
+  public void clearGate(final String gateName) throws RemoteException {
+    gates.remove(gateName);
+  }
+
+  @Override
   public void signalGate(final String gateName) throws RemoteException {
     gates.put(gateName, Boolean.TRUE);
   }


[11/13] geode git commit: Safe refactorings

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
index 709308b..33a7f52 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
@@ -106,8 +106,8 @@ public interface InternalCache extends Cache, Extensible<Cache> {
 
   boolean requiresNotificationFromPR(PartitionedRegion r);
 
-  RegionAttributes invokeRegionBefore(LocalRegion parent, String name, RegionAttributes attrs,
-      InternalRegionArguments internalRegionArgs);
+  <K, V> RegionAttributes<K, V> invokeRegionBefore(LocalRegion parent, String name,
+      RegionAttributes<K, V> attrs, InternalRegionArguments internalRegionArgs);
 
   void invokeRegionAfter(LocalRegion region);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index 45035d7..8c061b0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -5853,8 +5853,8 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       tx.txPutEntry(event, ifNew, false, false, null);
       return null;
     } else {
-      if (GemFireCacheImpl.internalBeforeNonTXBasicPut != null) {
-        GemFireCacheImpl.internalBeforeNonTXBasicPut.run();
+      if (DistTXState.internalBeforeNonTXBasicPut != null) {
+        DistTXState.internalBeforeNonTXBasicPut.run();
       }
 
       RegionEntry oldEntry = this.entries.basicPut(event, lastModified, ifNew, false, // ifOld

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java
index 5da63ad..13d8e18 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/FindRestEnabledServersFunction.java
@@ -34,6 +34,12 @@ import org.apache.geode.management.internal.RestAgent;
 
 public class FindRestEnabledServersFunction extends FunctionAdapter implements InternalEntity {
 
+  /**
+   * This property defines internal function that will get executed on each node to fetch active
+   * REST service endpoints (servers).
+   */
+  public static final String FIND_REST_ENABLED_SERVERS_FUNCTION_ID =
+      FindRestEnabledServersFunction.class.getName();
   private static final long serialVersionUID = 7851518767859544678L;
 
 
@@ -61,7 +67,7 @@ public class FindRestEnabledServersFunction extends FunctionAdapter implements I
   }
 
   public String getId() {
-    return GemFireCacheImpl.FIND_REST_ENABLED_SERVERS_FUNCTION_ID;
+    return FIND_REST_ENABLED_SERVERS_FUNCTION_ID;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/PersistenceAdvisorImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/PersistenceAdvisorImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/PersistenceAdvisorImpl.java
index 7e30141..fc95f0b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/PersistenceAdvisorImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/PersistenceAdvisorImpl.java
@@ -365,10 +365,10 @@ public class PersistenceAdvisorImpl implements PersistenceAdvisor {
 
   /**
    * Start listening for persistent view updates and apply any updates that have already happened.
-   * 
+   *
    * This method should be called after we have decided that there is no conflicting persistent
    * exception.
-   * 
+   *
    * Fix for bug 44045.
    */
   protected void beginUpdatingPersistentView() {
@@ -776,9 +776,9 @@ public class PersistenceAdvisorImpl implements PersistenceAdvisor {
   /**
    * Returns the member id of the member who has the latest copy of the persistent region. This may
    * be the local member ID if this member has the latest known copy.
-   * 
+   *
    * This method will block until the latest member is online.
-   * 
+   *
    * @throws ConflictingPersistentDataException if there are active members which are not based on
    *         the state that is persisted in this member.
    */

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
index a5f0fc2..b07ccba 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
@@ -1401,7 +1401,7 @@ public class CacheCreation implements InternalCache {
    * 
    * @see org.apache.geode.cache.Cache#getMembers(org.apache.geode.cache.Region)
    */
-  public Set<DistributedMember> getMembers(Region r) {
+  public Set<DistributedMember> getMembers(Region region) {
     return Collections.EMPTY_SET;
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
index d6a1efa..2b847d0 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
@@ -1270,7 +1270,6 @@ public class MemberMBeanBridge {
 
 
   /**
-   * 
    * @return the duration for which the member is up
    */
   public long getMemberUpTime() {
@@ -1278,25 +1277,23 @@ public class MemberMBeanBridge {
   }
 
   /**
-   * 
    * @return root region names
    */
   public String[] getRootRegionNames() {
-    Set<LocalRegion> listOfRootRegions = cache.rootRegions();
+    Set<Region<?, ?>> listOfRootRegions = cache.rootRegions();
     if (listOfRootRegions != null && listOfRootRegions.size() > 0) {
-      String[] regionStr = new String[listOfRootRegions.size()];
+      String[] regionNames = new String[listOfRootRegions.size()];
       int j = 0;
-      for (LocalRegion rg : listOfRootRegions) {
-        regionStr[j] = rg.getFullPath();
+      for (Region region : listOfRootRegions) {
+        regionNames[j] = region.getFullPath();
         j++;
       }
-      return regionStr;
+      return regionNames;
     }
     return ManagementConstants.NO_DATA_STRING;
   }
 
   /**
-   * 
    * @return Current GemFire version
    */
   public String getVersion() {

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/TXJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/TXJUnitTest.java b/geode-core/src/test/java/org/apache/geode/TXJUnitTest.java
index 54d9e50..598d67a 100644
--- a/geode-core/src/test/java/org/apache/geode/TXJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/TXJUnitTest.java
@@ -14,8 +14,68 @@
  */
 package org.apache.geode;
 
-import org.apache.geode.cache.*;
-import org.apache.geode.cache.query.*;
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Properties;
+import java.util.Set;
+
+import javax.transaction.Synchronization;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import org.apache.geode.cache.AttributesFactory;
+import org.apache.geode.cache.AttributesMutator;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheEvent;
+import org.apache.geode.cache.CacheException;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.CacheListener;
+import org.apache.geode.cache.CacheLoader;
+import org.apache.geode.cache.CacheLoaderException;
+import org.apache.geode.cache.CacheTransactionManager;
+import org.apache.geode.cache.CacheWriter;
+import org.apache.geode.cache.CacheWriterException;
+import org.apache.geode.cache.CommitConflictException;
+import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.DiskStoreFactory;
+import org.apache.geode.cache.EntryEvent;
+import org.apache.geode.cache.EntryExistsException;
+import org.apache.geode.cache.EntryNotFoundException;
+import org.apache.geode.cache.EvictionAction;
+import org.apache.geode.cache.EvictionAttributes;
+import org.apache.geode.cache.FailedSynchronizationException;
+import org.apache.geode.cache.LoaderHelper;
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.cache.PartitionAttributesFactory;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionEvent;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.cache.TimeoutException;
+import org.apache.geode.cache.TransactionEvent;
+import org.apache.geode.cache.TransactionException;
+import org.apache.geode.cache.TransactionId;
+import org.apache.geode.cache.TransactionListener;
+import org.apache.geode.cache.UnsupportedOperationInTransactionException;
+import org.apache.geode.cache.query.Index;
+import org.apache.geode.cache.query.IndexType;
+import org.apache.geode.cache.query.Query;
+import org.apache.geode.cache.query.QueryException;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.SelectResults;
 import org.apache.geode.cache.query.internal.index.IndexManager;
 import org.apache.geode.cache.util.CacheListenerAdapter;
 import org.apache.geode.cache.util.TransactionListenerAdapter;
@@ -23,34 +83,29 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.NanoTimer;
-import org.apache.geode.internal.cache.*;
+import org.apache.geode.internal.cache.AbstractRegion;
+import org.apache.geode.internal.cache.CachePerfStats;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.TXManagerImpl;
+import org.apache.geode.internal.cache.TXStateProxy;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.util.StopWatch;
 import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.junit.*;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-import javax.transaction.Synchronization;
-import java.util.*;
-
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.junit.Assert.*;
 
 /**
  * Tests basic transaction functionality
  *
  * @since GemFire 4.0
- *
  */
 @Category(IntegrationTest.class)
+@SuppressWarnings("deprecated")
 public class TXJUnitTest {
 
-  @Rule
-  public TestName testName = new TestName();
-
   private int cbCount;
   private TransactionEvent te;
+
   protected int listenerAfterCommit;
   protected int listenerAfterFailedCommit;
   protected int listenerAfterRollback;
@@ -58,16 +113,21 @@ public class TXJUnitTest {
   protected CacheTransactionManager txMgr;
 
   protected GemFireCacheImpl cache;
-  protected Region region;
+  protected Region<String, String> region;
+
+  @Rule
+  public TestName testName = new TestName();
 
   private boolean isPR() {
     return (this.region instanceof PartitionedRegion);
   }
 
   protected void createCache() throws Exception {
-    Properties p = new Properties();
-    p.setProperty(MCAST_PORT, "0"); // loner
-    this.cache = (GemFireCacheImpl) CacheFactory.create(DistributedSystem.connect(p));
+    Properties properties = new Properties();
+    properties.setProperty(MCAST_PORT, "0"); // loner
+
+    this.cache = (GemFireCacheImpl) CacheFactory.create(DistributedSystem.connect(properties));
+
     createRegion();
     this.txMgr = this.cache.getCacheTransactionManager();
     this.listenerAfterCommit = 0;
@@ -76,15 +136,13 @@ public class TXJUnitTest {
     this.listenerClose = 0;
   }
 
-  /**
-   * 
-   */
   protected void createRegion() throws Exception {
-    AttributesFactory af = new AttributesFactory();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
-    af.setConcurrencyChecksEnabled(false); // test validation expects this behavior
-    af.setIndexMaintenanceSynchronous(true);
-    this.region = this.cache.createRegion("TXJUnitTest", af.create());
+    AttributesFactory<String, String> attributesFactory = new AttributesFactory<>();
+    attributesFactory.setScope(Scope.DISTRIBUTED_NO_ACK);
+    attributesFactory.setConcurrencyChecksEnabled(false); // test validation expects this behavior
+    attributesFactory.setIndexMaintenanceSynchronous(true);
+
+    this.region = this.cache.createRegion(getClass().getSimpleName(), attributesFactory.create());
   }
 
   protected void closeCache() {
@@ -104,12 +162,12 @@ public class TXJUnitTest {
   }
 
   @Before
-  public void setUp() throws Exception {
+  public void setUpTXJUnitTest() throws Exception {
     createCache();
   }
 
   @After
-  public void tearDown() throws Exception {
+  public void tearDownTXJUnitTest() throws Exception {
     closeCache();
   }
 
@@ -358,31 +416,37 @@ public class TXJUnitTest {
     final CachePerfStats stats = this.cache.getCachePerfStats();
     int txCommitChanges;
     TransactionId myTxId;
-    AttributesFactory af = new AttributesFactory();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
-    Region reg1 = this.region;
-    Region reg2 = this.cache.createRegion(getUniqueName(), af.create());
+
+    AttributesFactory<String, String> attributesFactory = new AttributesFactory<>();
+    attributesFactory.setScope(Scope.DISTRIBUTED_NO_ACK);
+
+    Region<String, String> reg1 = this.region;
+    Region<String, String> reg2 =
+        this.cache.createRegion(getUniqueName(), attributesFactory.create());
 
     this.txMgr.setListener(new TransactionListener() {
+      @Override
       public void afterCommit(TransactionEvent event) {
         listenerAfterCommit = 1;
         te = event;
       }
 
+      @Override
       public void afterFailedCommit(TransactionEvent event) {
         listenerAfterFailedCommit = 1;
         te = event;
       }
 
+      @Override
       public void afterRollback(TransactionEvent event) {
         listenerAfterRollback = 1;
         te = event;
       }
 
+      @Override
       public void close() {
         listenerClose = 1;
       }
-
     });
 
     // see if commits work
@@ -406,12 +470,11 @@ public class TXJUnitTest {
     assertEquals("value2", reg2.get("key2"));
     assertEquals(txCommitChanges + 2, stats.getTxCommitChanges());
     {
-      Collection creates = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> creates = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(2, creates.size());
-      Iterator it = creates.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : creates) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1 || ev.getRegion() == reg2);
         if (ev.getRegion() == reg1) {
@@ -442,28 +505,31 @@ public class TXJUnitTest {
   @Test
   public void testTxEvent() throws CacheException {
     TransactionId myTxId;
-    Region reg1 = this.region;
+    Region<String, String> reg1 = this.region;
 
     this.txMgr.setListener(new TransactionListener() {
+      @Override
       public void afterCommit(TransactionEvent event) {
         listenerAfterCommit = 1;
         te = event;
       }
 
+      @Override
       public void afterFailedCommit(TransactionEvent event) {
         listenerAfterFailedCommit = 1;
         te = event;
       }
 
+      @Override
       public void afterRollback(TransactionEvent event) {
         listenerAfterRollback = 1;
         te = event;
       }
 
+      @Override
       public void close() {
         listenerClose = 1;
       }
-
     });
 
     // make sure each operation has the correct transaction event
@@ -479,12 +545,11 @@ public class TXJUnitTest {
     {
       Cache teCache = this.te.getCache();
       assertEquals(teCache, this.cache);
-      Collection creates = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> creates = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, creates.size());
-      Iterator it = creates.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : creates) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -515,12 +580,11 @@ public class TXJUnitTest {
     {
       Cache teCache = this.te.getCache();
       assertEquals(teCache, this.cache);
-      Collection creates = this.te.getPutEvents();
+      List<EntryEvent<?, ?>> creates = this.te.getPutEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, creates.size());
-      Iterator it = creates.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : creates) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -551,12 +615,11 @@ public class TXJUnitTest {
     {
       Cache teCache = this.te.getCache();
       assertEquals(teCache, this.cache);
-      Collection creates = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> creates = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, creates.size());
-      Iterator it = creates.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : creates) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -587,12 +650,11 @@ public class TXJUnitTest {
     {
       Cache teCache = this.te.getCache();
       assertEquals(teCache, this.cache);
-      Collection creates = this.te.getInvalidateEvents();
+      List<EntryEvent<?, ?>> creates = this.te.getInvalidateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, creates.size());
-      Iterator it = creates.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : creates) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -624,12 +686,11 @@ public class TXJUnitTest {
     {
       Cache teCache = this.te.getCache();
       assertEquals(teCache, this.cache);
-      Collection creates = this.te.getInvalidateEvents();
+      List<EntryEvent<?, ?>> creates = this.te.getInvalidateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, creates.size());
-      Iterator it = creates.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : creates) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -662,12 +723,11 @@ public class TXJUnitTest {
     {
       Cache teCache = this.te.getCache();
       assertEquals(teCache, this.cache);
-      Collection creates = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> creates = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, creates.size());
-      Iterator it = creates.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : creates) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -699,12 +759,11 @@ public class TXJUnitTest {
     {
       Cache teCache = this.te.getCache();
       assertEquals(teCache, this.cache);
-      Collection creates = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> creates = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, creates.size());
-      Iterator it = creates.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : creates) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -726,7 +785,7 @@ public class TXJUnitTest {
   }
 
   private static class CountingCallBackValidator {
-    ArrayList asserts;
+    List<Object> asserts;
     final String createWriterAssert = "create writer Assert";
     final String createListenerAssert = "create listener Assert";
     final String updateWriterAssert = "update writer Assert";
@@ -743,7 +802,7 @@ public class TXJUnitTest {
     CountingCallBackValidator(CountingCacheListener cl, CountingCacheWriter cw) {
       this.cl = cl;
       this.cw = cw;
-      this.asserts = new ArrayList(8);
+      this.asserts = new ArrayList<>(8);
     }
 
     void assertCreateWriterCnt(int cnt) {
@@ -753,7 +812,7 @@ public class TXJUnitTest {
     void assertCreateWriterCnt(int cnt, boolean remember) {
       if (remember) {
         this.asserts.add(createWriterAssert);
-        this.asserts.add(new Integer(cnt));
+        this.asserts.add(cnt);
       }
       assertEquals(cnt, this.cw.getBeforeCreateCalls());
     }
@@ -765,7 +824,7 @@ public class TXJUnitTest {
     void assertCreateListenerCnt(int cnt, boolean remember) {
       if (remember) {
         this.asserts.add(createListenerAssert);
-        this.asserts.add(new Integer(cnt));
+        this.asserts.add(cnt);
       }
       assertEquals(cnt, this.cl.getAfterCreateCalls());
     }
@@ -777,7 +836,7 @@ public class TXJUnitTest {
     void assertDestroyWriterCnt(int cnt, boolean remember) {
       if (remember) {
         this.asserts.add(destroyWriterAssert);
-        this.asserts.add(new Integer(cnt));
+        this.asserts.add(cnt);
       }
       assertEquals(cnt, this.cw.getBeforeDestroyCalls(false));
     }
@@ -789,7 +848,7 @@ public class TXJUnitTest {
     void assertDestroyListenerCnt(int cnt, boolean remember) {
       if (remember) {
         this.asserts.add(destroyListenerAssert);
-        this.asserts.add(new Integer(cnt));
+        this.asserts.add(cnt);
       }
       assertEquals(cnt, this.cl.getAfterDestroyCalls(false));
     }
@@ -801,7 +860,7 @@ public class TXJUnitTest {
     void assertLocalDestroyWriterCnt(int cnt, boolean remember) {
       if (remember) {
         this.asserts.add(localDestroyWriterAssert);
-        this.asserts.add(new Integer(cnt));
+        this.asserts.add(cnt);
       }
       assertEquals(0, this.cw.getBeforeDestroyCalls(true));
     }
@@ -813,7 +872,7 @@ public class TXJUnitTest {
     void assertLocalDestroyListenerCnt(int cnt, boolean remember) {
       if (remember) {
         this.asserts.add(localDestroyListenerAssert);
-        this.asserts.add(new Integer(cnt));
+        this.asserts.add(cnt);
       }
       assertEquals(cnt, this.cl.getAfterDestroyCalls(true));
     }
@@ -825,7 +884,7 @@ public class TXJUnitTest {
     void assertUpdateWriterCnt(int cnt, boolean remember) {
       if (remember) {
         this.asserts.add(updateWriterAssert);
-        this.asserts.add(new Integer(cnt));
+        this.asserts.add(cnt);
       }
       assertEquals(cnt, this.cw.getBeforeUpdateCalls());
     }
@@ -837,7 +896,7 @@ public class TXJUnitTest {
     void assertUpdateListenerCnt(int cnt, boolean remember) {
       if (remember) {
         this.asserts.add(updateListenerAssert);
-        this.asserts.add(new Integer(cnt));
+        this.asserts.add(cnt);
       }
       assertEquals(cnt, this.cl.getAfterUpdateCalls());
     }
@@ -849,7 +908,7 @@ public class TXJUnitTest {
     void assertInvalidateCnt(int cnt, boolean remember) {
       if (remember) {
         this.asserts.add(invalAssert);
-        this.asserts.add(new Integer(cnt));
+        this.asserts.add(cnt);
       }
       assertEquals(cnt, this.cl.getAfterInvalidateCalls());
     }
@@ -864,7 +923,7 @@ public class TXJUnitTest {
         assertTrue("CountingCallBackValidator reassert, did not have an associated count",
             assertItr.hasNext());
         count = (Integer) assertItr.next();
-        cnt = count.intValue();
+        cnt = count;
         if (assertType.equals(createWriterAssert)) {
           this.assertCreateWriterCnt(cnt, false);
         } else if (assertType.equals(createListenerAssert)) {
@@ -896,77 +955,86 @@ public class TXJUnitTest {
     }
   }
 
-  private static interface CountingCacheListener extends CacheListener {
-    public int getAfterCreateCalls();
+  private interface CountingCacheListener extends CacheListener {
+    int getAfterCreateCalls();
 
-    public int getAfterUpdateCalls();
+    int getAfterUpdateCalls();
 
-    public int getAfterInvalidateCalls();
+    int getAfterInvalidateCalls();
 
-    public int getAfterDestroyCalls(boolean fetchLocal);
+    int getAfterDestroyCalls(boolean fetchLocal);
 
-    public void reset();
+    void reset();
   }
 
-  private static interface CountingCacheWriter extends CacheWriter {
-    public int getBeforeCreateCalls();
+  private interface CountingCacheWriter extends CacheWriter {
+    int getBeforeCreateCalls();
 
-    public int getBeforeUpdateCalls();
+    int getBeforeUpdateCalls();
 
-    public int getBeforeDestroyCalls(boolean fetchLocal);
+    int getBeforeDestroyCalls(boolean fetchLocal);
 
-    public void reset();
+    void reset();
   }
 
   @Test
   public void testTxAlgebra() throws CacheException {
     TransactionId myTxId;
-    Region reg1 = this.region;
+    Region<String, String> reg1 = this.region;
 
     this.txMgr.setListener(new TransactionListener() {
+      @Override
       public void afterCommit(TransactionEvent event) {
         listenerAfterCommit = 1;
         te = event;
       }
 
+      @Override
       public void afterFailedCommit(TransactionEvent event) {
         listenerAfterFailedCommit = 1;
         te = event;
       }
 
+      @Override
       public void afterRollback(TransactionEvent event) {
         listenerAfterRollback = 1;
         te = event;
       }
 
+      @Override
       public void close() {
         listenerClose = 1;
       }
-
     });
-    AttributesMutator mutator = this.region.getAttributesMutator();
+    AttributesMutator<String, String> mutator = this.region.getAttributesMutator();
     CountingCacheListener cntListener = new CountingCacheListener() {
       volatile int aCreateCalls, aUpdateCalls, aInvalidateCalls, aDestroyCalls, aLocalDestroyCalls;
 
+      @Override
       public void close() {}
 
+      @Override
       public void reset() {
         this.aCreateCalls = this.aUpdateCalls =
             this.aInvalidateCalls = this.aDestroyCalls = this.aLocalDestroyCalls = 0;
       }
 
+      @Override
       public void afterCreate(EntryEvent e) {
         ++this.aCreateCalls;
       }
 
+      @Override
       public void afterUpdate(EntryEvent e) {
         ++this.aUpdateCalls;
       }
 
+      @Override
       public void afterInvalidate(EntryEvent e) {
         ++this.aInvalidateCalls;
       }
 
+      @Override
       public void afterDestroy(EntryEvent e) {
         if (e.isDistributed()) {
           ++this.aDestroyCalls;
@@ -975,34 +1043,43 @@ public class TXJUnitTest {
         }
       }
 
+      @Override
       public void afterRegionInvalidate(RegionEvent e) {
         fail("Unexpected afterRegionInvalidate in testTxAlgebra");
       }
 
+      @Override
       public void afterRegionDestroy(RegionEvent e) {
         if (!e.getOperation().isClose()) {
           fail("Unexpected afterRegionDestroy in testTxAlgebra");
         }
       }
 
+      @Override
       public void afterRegionClear(RegionEvent event) {}
 
+      @Override
       public void afterRegionCreate(RegionEvent event) {}
 
+      @Override
       public void afterRegionLive(RegionEvent event) {}
 
+      @Override
       public int getAfterCreateCalls() {
         return this.aCreateCalls;
       }
 
+      @Override
       public int getAfterUpdateCalls() {
         return this.aUpdateCalls;
       }
 
+      @Override
       public int getAfterInvalidateCalls() {
         return this.aInvalidateCalls;
       }
 
+      @Override
       public int getAfterDestroyCalls(boolean fetchLocal) {
         return fetchLocal ? this.aLocalDestroyCalls : this.aDestroyCalls;
       }
@@ -1011,40 +1088,50 @@ public class TXJUnitTest {
     CountingCacheWriter cntWriter = new CountingCacheWriter() {
       int bCreateCalls, bUpdateCalls, bDestroyCalls, bLocalDestroyCalls;
 
+      @Override
       public void close() {}
 
+      @Override
       public void reset() {
         this.bCreateCalls = this.bUpdateCalls = this.bDestroyCalls = this.bLocalDestroyCalls = 0;
       }
 
+      @Override
       public void beforeCreate(EntryEvent e) {
         ++this.bCreateCalls;
       }
 
+      @Override
       public void beforeUpdate(EntryEvent e) {
         ++this.bUpdateCalls;
       }
 
+      @Override
       public void beforeDestroy(EntryEvent e) {
         ++this.bDestroyCalls;
       }
 
+      @Override
       public void beforeRegionDestroy(RegionEvent e) {
         fail("Unexpected beforeRegionDestroy in testTxAlgebra");
       }
 
+      @Override
       public void beforeRegionClear(RegionEvent e) {
         fail("Unexpected beforeRegionClear in testTxAlgebra");
       }
 
+      @Override
       public int getBeforeCreateCalls() {
         return this.bCreateCalls;
       }
 
+      @Override
       public int getBeforeUpdateCalls() {
         return this.bUpdateCalls;
       }
 
+      @Override
       public int getBeforeDestroyCalls(boolean fetchLocal) {
         return fetchLocal ? this.bLocalDestroyCalls : this.bDestroyCalls;
       }
@@ -1082,12 +1169,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getInvalidateEvents().size());
     assertEquals(0, this.te.getDestroyEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1127,12 +1213,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getInvalidateEvents().size());
     assertEquals(0, this.te.getDestroyEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1151,7 +1236,7 @@ public class TXJUnitTest {
     }
     reg1.localDestroy("key1");
 
-    // @todo mitch implement the following
+    // TODO: mitch implement the following
     // check LI + DI -> NOOP
     // check DI + LI -> NOOP
     // check DI + DI -> NOOP
@@ -1160,7 +1245,6 @@ public class TXJUnitTest {
     // check C + DD -> NOOP
     callbackVal.reset();
     this.txMgr.begin();
-    myTxId = this.txMgr.getTransactionId();
     reg1.create("key1", "value0");
     callbackVal.assertCreateWriterCnt(1);
     reg1.destroy("key1");
@@ -1198,12 +1282,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1253,12 +1336,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1280,7 +1362,6 @@ public class TXJUnitTest {
     // Check C + LI + LD -> NOOP
     callbackVal.reset();
     this.txMgr.begin();
-    myTxId = this.txMgr.getTransactionId();
     reg1.create("key1", "value1");
     callbackVal.assertCreateWriterCnt(1);
     reg1.localInvalidate("key1");
@@ -1302,7 +1383,6 @@ public class TXJUnitTest {
     // Check C + LI + DD -> NOOP
     callbackVal.reset();
     this.txMgr.begin();
-    myTxId = this.txMgr.getTransactionId();
     reg1.create("key1", "value1");
     callbackVal.assertCreateWriterCnt(1);
     reg1.localInvalidate("key1");
@@ -1324,7 +1404,6 @@ public class TXJUnitTest {
     // check C + LD -> NOOP
     callbackVal.reset();
     this.txMgr.begin();
-    myTxId = this.txMgr.getTransactionId();
     reg1.create("key1", "value0");
     callbackVal.assertCreateWriterCnt(1);
     reg1.localDestroy("key1");
@@ -1388,12 +1467,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1433,12 +1511,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1486,12 +1563,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getPutEvents();
+      List<EntryEvent<?, ?>> events = this.te.getPutEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1532,12 +1608,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getInvalidateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getInvalidateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1577,12 +1652,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getInvalidateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> events = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1622,12 +1696,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getInvalidateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getInvalidateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1677,12 +1750,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getPutEvents();
+      List<EntryEvent<?, ?>> events = this.te.getPutEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1724,12 +1796,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getCreateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> events = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1769,12 +1840,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getCreateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> events = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1813,12 +1883,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getInvalidateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> events = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1885,12 +1954,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1931,12 +1999,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -1981,12 +2048,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getPutEvents();
+      List<EntryEvent<?, ?>> events = this.te.getPutEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2024,12 +2090,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getInvalidateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> events = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2067,12 +2132,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getInvalidateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> events = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2142,12 +2206,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2190,12 +2253,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2241,12 +2303,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getPutEvents();
+      List<EntryEvent<?, ?>> events = this.te.getPutEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2285,12 +2346,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getInvalidateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> events = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2328,12 +2388,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getInvalidateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getDestroyEvents();
+      List<EntryEvent<?, ?>> events = this.te.getDestroyEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2403,12 +2462,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2451,12 +2509,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2501,12 +2558,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getCreateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getInvalidateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getInvalidateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2549,12 +2605,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getCreateEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getInvalidateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getInvalidateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2594,12 +2649,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2639,12 +2693,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2684,12 +2737,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2729,12 +2781,11 @@ public class TXJUnitTest {
     assertEquals(0, this.te.getDestroyEvents().size());
     assertEquals(1, this.te.getEvents().size());
     {
-      Collection events = this.te.getCreateEvents();
+      List<EntryEvent<?, ?>> events = this.te.getCreateEvents();
       assertEquals(myTxId, this.te.getTransactionId());
       assertEquals(1, events.size());
-      Iterator it = events.iterator();
-      while (it.hasNext()) {
-        EntryEvent ev = (EntryEvent) it.next();
+
+      for (EntryEvent ev : events) {
         assertEquals(myTxId, ev.getTransactionId());
         assertTrue(ev.getRegion() == reg1);
         assertEquals("key1", ev.getKey());
@@ -2898,25 +2949,28 @@ public class TXJUnitTest {
   public void testListener() {
     assertTrue(this.txMgr.getListener() == null);
     TransactionListener oldListener = this.txMgr.setListener(new TransactionListener() {
+      @Override
       public void afterCommit(TransactionEvent event) {
         listenerAfterCommit = 1;
         te = event;
       }
 
+      @Override
       public void afterFailedCommit(TransactionEvent event) {
         listenerAfterFailedCommit = 1;
         te = event;
       }
 
+      @Override
       public void afterRollback(TransactionEvent event) {
         listenerAfterRollback = 1;
         te = event;
       }
 
+      @Override
       public void close() {
         listenerClose = 1;
       }
-
     });
     assertTrue(oldListener == null);
     this.txMgr.begin();
@@ -2949,20 +3003,24 @@ public class TXJUnitTest {
 
     assertEquals(0, this.listenerClose);
     oldListener = this.txMgr.setListener(new TransactionListener() {
+      @Override
       public void afterCommit(TransactionEvent event) {
         listenerAfterCommit = 2;
         te = event;
       }
 
+      @Override
       public void afterFailedCommit(TransactionEvent event) {
         listenerAfterFailedCommit = 2;
       }
 
+      @Override
       public void afterRollback(TransactionEvent event) {
         listenerAfterRollback = 2;
         te = event;
       }
 
+      @Override
       public void close() {
         listenerClose = 2;
       }
@@ -2988,57 +3046,70 @@ public class TXJUnitTest {
   @Test
   public void testNoCallbacksOnRollback() throws CacheException {
     // install listeners
-    AttributesMutator mutator = this.region.getAttributesMutator();
-    mutator.setCacheListener(new CacheListenerAdapter() {
+    AttributesMutator<String, String> mutator = this.region.getAttributesMutator();
+    mutator.setCacheListener(new CacheListenerAdapter<String, String>() {
+      @Override
       public void close() {
         cbCount++;
       }
 
+      @Override
       public void afterCreate(EntryEvent event) {
         cbCount++;
       }
 
+      @Override
       public void afterUpdate(EntryEvent event) {
         cbCount++;
       }
 
+      @Override
       public void afterInvalidate(EntryEvent event) {
         cbCount++;
       }
 
+      @Override
       public void afterDestroy(EntryEvent event) {
         cbCount++;
       }
 
+      @Override
       public void afterRegionInvalidate(RegionEvent event) {
         cbCount++;
       }
 
+      @Override
       public void afterRegionDestroy(RegionEvent event) {
         cbCount++;
       }
     });
-    mutator.setCacheWriter(new CacheWriter() {
+    mutator.setCacheWriter(new CacheWriter<String, String>() {
+      @Override
       public void close() {
         cbCount++;
       }
 
+      @Override
       public void beforeUpdate(EntryEvent event) throws CacheWriterException {
         cbCount++;
       }
 
+      @Override
       public void beforeCreate(EntryEvent event) throws CacheWriterException {
         cbCount++;
       }
 
+      @Override
       public void beforeDestroy(EntryEvent event) throws CacheWriterException {
         cbCount++;
       }
 
+      @Override
       public void beforeRegionDestroy(RegionEvent event) throws CacheWriterException {
         cbCount++;
       }
 
+      @Override
       public void beforeRegionClear(RegionEvent event) throws CacheWriterException {
         cbCount++;
       }
@@ -3094,10 +3165,9 @@ public class TXJUnitTest {
     this.region.localDestroy("key1");
   }
 
-  //
-  // TXCallBackValidator is a container for holding state for validating Cache
-  // callbacks
-  //
+  /**
+   * TXCallBackValidator is a container for holding state for validating Cache callbacks
+   */
   private class TXCallBackValidator {
     boolean passedValidation;
     boolean suspendValidation;
@@ -3116,8 +3186,9 @@ public class TXJUnitTest {
     boolean isInvalidate;
     Object callBackArg;
 
-    // EntryEvent, CallCount validator for
-    // callbacks (CacheWriter, CacheListener
+    /**
+     * EntryEvent, CallCount validator for callbacks (CacheWriter, CacheListener
+     */
     boolean validate(EntryEvent event, int cnt) {
       if (this.isSuspendValidation()) {
         return true;
@@ -3135,8 +3206,7 @@ public class TXJUnitTest {
       assertNotNull(event.getRegion().getCache());
       assertNotNull(event.getRegion().getCache().getCacheTransactionManager());
       assertEquals(this.getTXId(), event.getTransactionId());
-      // assertIndexDetailsEquals(event.getTransactionId(),
-      // event.getRegion().getCache().getCacheTransactionManager().getTransactionId(), );
+
       if (!isPR())
         assertEquals("IsDistributed Assertion!", this.isDistributed(), event.isDistributed());
       assertEquals(this.getKey(), event.getKey());
@@ -3161,12 +3231,6 @@ public class TXJUnitTest {
       return oldVal;
     }
 
-    // int getCount() {return this.callCount;}
-    // int setCount(int newVal) {
-    // int oldVal = this.callCount;
-    // this.callCount = newVal;
-    // return oldVal;
-    // }
     void setKey(Object key) {
       this.key = key;
     }
@@ -3277,35 +3341,39 @@ public class TXJUnitTest {
       return this.passedValidation;
     }
   }
-  private static interface ValidatableCacheListener extends CacheListener {
-    public void setValidator(TXCallBackValidator v);
 
-    public void validate();
+  private interface ValidatableCacheListener extends CacheListener {
+    void setValidator(TXCallBackValidator v);
 
-    public void validateNoEvents();
+    void validate();
 
-    public void reset();
+    void validateNoEvents();
 
-    public void setExpectedCount(int count);
+    void reset();
 
-    public int getCallCount();
+    void setExpectedCount(int count);
+
+    int getCallCount();
   }
-  private static interface ValidatableCacheWriter extends CacheWriter {
-    public void setValidator(TXCallBackValidator v);
 
-    public int getCallCount();
+  private interface ValidatableCacheWriter extends CacheWriter {
+    void setValidator(TXCallBackValidator v);
+
+    int getCallCount();
 
-    public void localDestroyMakeup(int count);
+    void localDestroyMakeup(int count);
 
-    public void validate();
+    void validate();
 
-    public void reset();
+    void reset();
 
-    public void validateNoEvents();
+    void validateNoEvents();
   }
 
-  // Test to make sure CacheListener callbacks are called in place with
-  // the CacheEvents properly constructed
+  /**
+   * Test to make sure CacheListener callbacks are called in place with the CacheEvents properly
+   * constructed
+   */
   @Test
   public void testCacheCallbacks() throws CacheException {
     final String key1 = "Key1";
@@ -3313,7 +3381,7 @@ public class TXJUnitTest {
     final String value2 = "value2";
     final String callBackArg = "call back arg";
     // install listeners
-    AttributesMutator mutator = this.region.getAttributesMutator();
+    AttributesMutator<String, String> mutator = this.region.getAttributesMutator();
 
     TXCallBackValidator cbv = new TXCallBackValidator();
 
@@ -3324,20 +3392,24 @@ public class TXJUnitTest {
       int prevCallCount;
       EntryEvent lastEvent;
 
+      @Override
       public void validate() {
         this.v.validate(this.lastEvent, this.callCount);
       }
 
-      public void validate(EntryEvent event) {
+      void validate(EntryEvent event) {
         this.v.validate(event, ++this.callCount);
       }
 
+      @Override
       public void setValidator(TXCallBackValidator v) {
         this.v = v;
       }
 
+      @Override
       public void close() {}
 
+      @Override
       public void afterCreate(EntryEvent event) {
         lastEvent = event;
         if (this.v.isSuspendValidation()) {
@@ -3352,6 +3424,7 @@ public class TXJUnitTest {
         this.v.setPassedValidation(true);
       }
 
+      @Override
       public void afterUpdate(EntryEvent event) {
         lastEvent = event;
         if (this.v.isSuspendValidation()) {
@@ -3366,6 +3439,7 @@ public class TXJUnitTest {
         this.v.setPassedValidation(true);
       }
 
+      @Override
       public void afterInvalidate(EntryEvent event) {
         lastEvent = event;
         if (this.v.isSuspendValidation()) {
@@ -3373,13 +3447,14 @@ public class TXJUnitTest {
         }
         validate(event);
         this.v.setPassedValidation(false);
-        assertTrue("IsInvaldiate Assertion!", this.v.isInvalidate());
+        assertTrue("IsInvalidate Assertion!", this.v.isInvalidate());
         assertTrue(event.getRegion().containsKey(this.v.getKey()));
         assertTrue(!event.getRegion().containsValueForKey(this.v.getKey()));
         assertNull(event.getRegion().getEntry(event.getKey()).getValue());
         this.v.setPassedValidation(true);
       }
 
+      @Override
       public void afterDestroy(EntryEvent event) {
         lastEvent = event;
         if (this.v.isSuspendValidation()) {
@@ -3394,38 +3469,45 @@ public class TXJUnitTest {
         this.v.setPassedValidation(true);
       }
 
+      @Override
       public void afterRegionInvalidate(RegionEvent event) {
-        fail("Unexpected invokation of afterRegionInvalidate");
+        fail("Unexpected invocation of afterRegionInvalidate");
       }
 
+      @Override
       public void afterRegionDestroy(RegionEvent event) {
         if (!event.getOperation().isClose()) {
-          fail("Unexpected invokation of afterRegionDestroy");
+          fail("Unexpected invocation of afterRegionDestroy");
         }
       }
 
-      public void afterRegionClear(RegionEvent event) {
-
-      }
+      @Override
+      public void afterRegionClear(RegionEvent event) {}
 
+      @Override
       public void afterRegionCreate(RegionEvent event) {}
 
+      @Override
       public void afterRegionLive(RegionEvent event) {}
 
+      @Override
       public void reset() {
         lastEvent = null;
         prevCallCount = callCount;
       }
 
+      @Override
       public void validateNoEvents() {
         assertNull("Did not expect listener callback", lastEvent);
         assertEquals(prevCallCount, callCount);
       }
 
+      @Override
       public void setExpectedCount(int count) {
         callCount = count;
       }
 
+      @Override
       public int getCallCount() {
         return callCount;
       }
@@ -3441,28 +3523,34 @@ public class TXJUnitTest {
       int prevCallCount;
       EntryEvent lastEvent;
 
+      @Override
       public int getCallCount() {
         return this.callCount;
       }
 
+      @Override
       public void localDestroyMakeup(int count) {
         this.callCount += count;
       }
 
+      @Override
       public void validate() {
         this.v.validate(this.lastEvent, this.callCount);
       }
 
-      public void validate(EntryEvent event) {
+      void validate(EntryEvent event) {
         this.v.validate(event, ++this.callCount);
       }
 
+      @Override
       public void setValidator(TXCallBackValidator v) {
         this.v = v;
       }
 
+      @Override
       public void close() {}
 
+      @Override
       public void beforeCreate(EntryEvent event) {
         lastEvent = event;
         if (this.v.isSuspendValidation()) {
@@ -3477,6 +3565,7 @@ public class TXJUnitTest {
         this.v.setPassedValidation(true);
       }
 
+      @Override
       public void beforeUpdate(EntryEvent event) {
         lastEvent = event;
         if (this.v.isSuspendValidation()) {
@@ -3486,11 +3575,12 @@ public class TXJUnitTest {
         this.v.setPassedValidation(false);
         assertTrue("IsUpdate Assertion!", this.v.isUpdate());
         assertTrue(event.getRegion().containsKey(this.v.getKey()));
-        // Can not assert the following line, as the value being update may be invalide
+        // Can not assert the following line, as the value being update may be invalid
         // assertTrue(event.getRegion().containsValueForKey(this.v.getKey()));
         this.v.setPassedValidation(true);
       }
 
+      @Override
       public void beforeDestroy(EntryEvent event) {
         lastEvent = event;
         if (this.v.isSuspendValidation()) {
@@ -3503,19 +3593,23 @@ public class TXJUnitTest {
         this.v.setPassedValidation(true);
       }
 
+      @Override
       public void beforeRegionDestroy(RegionEvent event) {
         fail("Unexpected invocation of beforeRegionDestroy");
       }
 
+      @Override
       public void beforeRegionClear(RegionEvent event) {
         fail("Unexpected invocation of beforeRegionClear");
       }
 
+      @Override
       public void reset() {
         lastEvent = null;
         prevCallCount = callCount;
       }
 
+      @Override
       public void validateNoEvents() {
         assertNull("Did not expect a writer event", lastEvent);
         assertEquals(prevCallCount, callCount);
@@ -3528,10 +3622,12 @@ public class TXJUnitTest {
     mutator.setCacheLoader(new CacheLoader() {
       int count = 0;
 
+      @Override
       public Object load(LoaderHelper helper) throws CacheLoaderException {
-        return new Integer(count++);
+        return count++;
       }
 
+      @Override
       public void close() {}
     });
 
@@ -3738,7 +3834,7 @@ public class TXJUnitTest {
 
     // Create load Event tests
     int loaderValCheck = 0;
-    cbv.setNewValue(new Integer(loaderValCheck++), false);
+    cbv.setNewValue(loaderValCheck++, false);
     cbv.setCallBackArg(null);
     cbv.setOldValue(null, false);
     cbv.setIsDistributed(true);
@@ -3759,7 +3855,7 @@ public class TXJUnitTest {
     vCl.reset();
     this.txMgr.begin();
     cbv.setTXId(txMgr.getTransactionId());
-    cbv.setNewValue(new Integer(loaderValCheck++), false);
+    cbv.setNewValue(loaderValCheck++, false);
     cbv.setExpectedCount(appCallCount++);
     this.region.get(key1);
     this.txMgr.rollback();
@@ -3770,7 +3866,7 @@ public class TXJUnitTest {
 
     this.txMgr.begin();
     cbv.setTXId(txMgr.getTransactionId());
-    cbv.setNewValue(new Integer(loaderValCheck++), false);
+    cbv.setNewValue(loaderValCheck++, false);
     cbv.setExpectedCount(appCallCount++);
     this.region.get(key1);
     vCw.validate();
@@ -3789,7 +3885,7 @@ public class TXJUnitTest {
     cbv.suspendValidation(false);
     assertTrue(this.region.containsKey(key1));
     assertTrue(!this.region.containsValueForKey(key1));
-    cbv.setNewValue(new Integer(loaderValCheck++), false);
+    cbv.setNewValue(loaderValCheck++, false);
     cbv.setOldValue(null, false);
     cbv.setIsDistributed(true);
     cbv.setCallBackArg(null);
@@ -3812,7 +3908,7 @@ public class TXJUnitTest {
     this.txMgr.begin();
     cbv.setTXId(txMgr.getTransactionId());
     cbv.setExpectedCount(appCallCount++);
-    cbv.setNewValue(new Integer(loaderValCheck++), false);
+    cbv.setNewValue(loaderValCheck++, false);
     this.region.get(key1);
     vCw.validate();
     vCw.reset();
@@ -3827,7 +3923,7 @@ public class TXJUnitTest {
     this.txMgr.begin();
     cbv.setTXId(txMgr.getTransactionId());
     cbv.setExpectedCount(appCallCount++);
-    cbv.setNewValue(new Integer(loaderValCheck++), false);
+    cbv.setNewValue(loaderValCheck++, false);
     this.region.get(key1);
     this.txMgr.rollback();
     assertTrue("TX Invalidate Validation Assertion", cbv.passedValidation());
@@ -3900,9 +3996,9 @@ public class TXJUnitTest {
 
   @Test
   public void testCollections() throws CacheException {
-    Region reg1 = this.region;
+    Region<String, String> reg1 = this.region;
 
-    checkSubRegionCollecection(reg1);
+    checkSubRegionCollection(reg1);
 
     {
       Collection nonTxKeys = reg1.keySet();
@@ -3954,28 +4050,20 @@ public class TXJUnitTest {
         assertTrue(!txIt.hasNext());
       }
       reg1.invalidate("key1");
-      // assertIndexDetailsEquals(0, nonTxKeys.size());
       assertEquals(1, txKeys.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");
-      // assertIndexDetailsEquals(0, nonTxKeys.size());
       assertEquals(3, txKeys.size());
-      // assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(2, txValues.size());
       reg1.put("key1", "value1");
-      // assertIndexDetailsEquals(0, nonTxKeys.size());
       assertEquals(3, txKeys.size());
-      // assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(3, txValues.size());
       reg1.localInvalidate("key2");
-      // assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(2, txValues.size());
       reg1.invalidate("key1");
-      // assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(1, txValues.size());
       reg1.destroy("key2");
       reg1.destroy("key3");
@@ -4010,11 +4098,9 @@ public class TXJUnitTest {
       txIt.hasNext();
     }
     {
-      // Collection nonTxValues = reg1.values();
       this.txMgr.begin();
       reg1.create("key1", "value1");
       Collection txValues = reg1.values();
-      // assertIndexDetailsEquals(0, nonTxValues.size());
       assertEquals(1, txValues.size());
       assertTrue(txValues.contains("value1"));
       {
@@ -4084,7 +4170,6 @@ public class TXJUnitTest {
       assertEquals(1, txValues.size());
       assertTrue(txValues.iterator().hasNext());
       assertEquals("txValue1", txValues.iterator().next());
-      // assertIndexDetailsEquals(0, nonTxValues.size());
       // non-TX collections can now be used in a transactional context
       try {
         nonTxValues.iterator().hasNext();
@@ -4103,18 +4188,23 @@ public class TXJUnitTest {
     }
   }
 
-  /**
-   * @param reg1
-   */
-  protected void checkSubRegionCollecection(Region reg1) {
-    AttributesFactory af = new AttributesFactory();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
-    Region sub1 = this.region.createSubregion("collectionSub1", af.create());
-    af = new AttributesFactory();
-    Region sub2 = this.region.createSubregion("collectionSub2", af.create());
-    af = new AttributesFactory();
-    af.setScope(Scope.LOCAL);
-    Region sub2_1 = sub2.createSubregion("collectionSub2_1", af.create());
+  protected void checkSubRegionCollection(Region<String, String> reg1) {
+    AttributesFactory<String, String> attributesFactory = new AttributesFactory<>();
+    attributesFactory.setScope(Scope.DISTRIBUTED_NO_ACK);
+
+    Region<String, String> sub1 =
+        this.region.createSubregion("collectionSub1", attributesFactory.create());
+
+    attributesFactory = new AttributesFactory<>();
+
+    Region<String, String> sub2 =
+        this.region.createSubregion("collectionSub2", attributesFactory.create());
+
+    attributesFactory = new AttributesFactory<>();
+    attributesFactory.setScope(Scope.LOCAL);
+
+    Region<String, String> sub2_1 =
+        sub2.createSubregion("collectionSub2_1", attributesFactory.create());
 
     checkCollectionSize(0);
     try {
@@ -4303,7 +4393,6 @@ public class TXJUnitTest {
     checkCollectionSize(2, 3);
     sub1.destroyRegion();
     checkCollectionSize(2);
-    // this.txMgr.rollback();
 
     reg1.localDestroy("key1");
     reg1.localDestroy("key3");
@@ -4312,18 +4401,20 @@ public class TXJUnitTest {
 
   @Test
   public void testLoader() throws CacheException {
-    LocalRegion reg1 = (LocalRegion) this.region;
-    AttributesMutator mutator = reg1.getAttributesMutator();
-    mutator.setCacheLoader(new CacheLoader() {
+    AttributesMutator<String, String> mutator = this.region.getAttributesMutator();
+    mutator.setCacheLoader(new CacheLoader<String, String>() {
       int count = 0;
 
-      public Object load(LoaderHelper helper) throws CacheLoaderException {
+      @Override
+      public String load(LoaderHelper helper) throws CacheLoaderException {
         count++;
         return "LV " + count;
       }
 
+      @Override
       public void close() {}
     });
+    LocalRegion reg1 = (LocalRegion) this.region;
     if (isPR())
       ((PartitionedRegion) reg1).setHaveCacheLoader();
     assertTrue(!reg1.containsKey("key1"));
@@ -4595,26 +4686,26 @@ public class TXJUnitTest {
     final CachePerfStats stats = this.cache.getCachePerfStats();
 
     class statsValidator {
-      long txSuccessLifeTime;
-      long txFailedLifeTime;
-      long txRollbackLifeTime;
-      int txCommits;
-      int txFailures;
-      int txRollbacks;
-      long txCommitTime;
-      long txFailureTime;
-      long txRollbackTime;
-      int txCommitChanges;
-      int txFailureChanges;
-      int txRollbackChanges;
-
-      CachePerfStats stats;
-
-      statsValidator(CachePerfStats stats) {
+      private long txSuccessLifeTime;
+      private long txFailedLifeTime;
+      private long txRollbackLifeTime;
+      private int txCommits;
+      private int txFailures;
+      private int txRollbacks;
+      private long txCommitTime;
+      private long txFailureTime;
+      private long txRollbackTime;
+      private int txCommitChanges;
+      private int txFailureChanges;
+      private int txRollbackChanges;
+
+      private CachePerfStats stats;
+
+      private statsValidator(CachePerfStats stats) {
         this.stats = stats;
       }
 
-      void reset() {
+      private void reset() {
         this.txSuccessLifeTime = this.stats.getTxSuccessLifeTime();
         this.txFailedLifeTime = this.stats.getTxFailedLifeTime();
         this.txRollbackLifeTime = this.stats.getTxRollbackLifeTime();
@@ -4629,55 +4720,55 @@ public class TXJUnitTest {
         this.txRollbackChanges = this.stats.getTxRollbackChanges();
       }
 
-      void setTxSuccessLifeTime(long txSuccessLifeTime) {
+      private void setTxSuccessLifeTime(long txSuccessLifeTime) {
         this.txSuccessLifeTime = txSuccessLifeTime;
       }
 
-      void setTxFailedLifeTime(long txFailedLifeTime) {
+      private void setTxFailedLifeTime(long txFailedLifeTime) {
         this.txFailedLifeTime = txFailedLifeTime;
       }
 
-      void setTxRollbackLifeTime(long txRollbackLifeTime) {
+      private void setTxRollbackLifeTime(long txRollbackLifeTime) {
         this.txRollbackLifeTime = txRollbackLifeTime;
       }
 
-      void setTxCommits(int txCommits) {
+      private void setTxCommits(int txCommits) {
         this.txCommits = txCommits;
       }
 
-      void setTxFailures(int txFailures) {
+      private void setTxFailures(int txFailures) {
         this.txFailures = txFailures;
       }
 
-      void setTxRollbacks(int txRollbacks) {
+      private void setTxRollbacks(int txRollbacks) {
         this.txRollbacks = txRollbacks;
       }
 
-      void setTxCommitTime(long txCommitTime) {
+      private void setTxCommitTime(long txCommitTime) {
         this.txCommitTime = txCommitTime;
       }
 
-      void setTxFailureTime(long txFailureTime) {
+      private void setTxFailureTime(long txFailureTime) {
         this.txFailureTime = txFailureTime;
       }
 
-      void setTxRollbackTime(long txRollbackTime) {
+      private void setTxRollbackTime(long txRollbackTime) {
         this.txRollbackTime = txRollbackTime;
       }
 
-      void setTxCommitChanges(int txCommitChanges) {
+      private void setTxCommitChanges(int txCommitChanges) {
         this.txCommitChanges = txCommitChanges;
       }
 
-      void setTxFailureChanges(int txFailureChanges) {
+      private void setTxFailureChanges(int txFailureChanges) {
         this.txFailureChanges = txFailureChanges;
       }
 
-      void setTxRollbackChanges(int txRollbackChanges) {
+      private void setTxRollbackChanges(int txRollbackChanges) {
         this.txRollbackChanges = txRollbackChanges;
       }
 
-      void assertValid() {
+      private void assertValid() {
         assertEquals(this.txRollbacks, this.stats.getTxRollbacks());
         assertEquals(this.txRollbackChanges, this.stats.getTxRollbackChanges());
         if (Boolean
@@ -4999,9 +5090,9 @@ public class TXJUnitTest {
   @Test
   public void testCheckNoTX() {
     {
-      AttributesFactory af = new AttributesFactory();
+      AttributesFactory<String, String> af = new AttributesFactory<>();
       af.setScope(Scope.GLOBAL);
-      Region gr = null;
+      Region<String, String> gr = null;
       try {
         gr = this.cache.createRegion("GLOBALTXTest", af.create());
       } catch (CacheException ex) {
@@ -5027,11 +5118,11 @@ public class TXJUnitTest {
     {
       DiskStoreFactory dsf = this.cache.createDiskStoreFactory();
       dsf.create("testCheckNoTX");
-      AttributesFactory af = new AttributesFactory();
+      AttributesFactory<String, String> af = new AttributesFactory<>();
       af.setScope(Scope.LOCAL);
       af.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
       af.setDiskStoreName("testCheckNoTX");
-      Region dr = null;
+      Region<String, String> dr = null;
       try {
         dr = this.cache.createRegion("DiskTXTest", af.create());
       } catch (CacheException ex) {
@@ -5568,13 +5659,8 @@ public class TXJUnitTest {
       fail("expected CommitConflictException");
     } catch (TransactionException ex) {
     }
-    // this.region is now destroyed
-
   }
 
-  /**
-   * @param txMgrImpl
-   */
   protected void checkUserAttributeConflict(final CacheTransactionManager txMgrImpl) {
     { // now check entry user attribute conflict checking
       this.region.put("key1", "value0");
@@ -5684,7 +5770,6 @@ public class TXJUnitTest {
       assertEquals(0, te.getEvents().size());
       this.region.destroy("key1");
 
-
       // now make sure that multiple invalidates of same entry are a single change
       txRollbackChanges = stats.getTxRollbackChanges();
       this.region.create("key1", "value1");
@@ -5740,7 +5825,7 @@ public class TXJUnitTest {
     }
   }
 
-  final static void clearRegion(Region r) throws TimeoutException {
+  private static void clearRegion(Region r) throws TimeoutException {
     Iterator kI = r.keySet().iterator();
     try {
       while (kI.hasNext()) {
@@ -5751,13 +5836,13 @@ public class TXJUnitTest {
     }
   }
 
-  final static int LRUENTRY_NULL = 0;
-  final static int LRUENTRY_STRING = 1;
-  final static int LRUENTRY_INTEGER = 2;
-  final static int LRUENTRY_LONG = 3;
-  final static int LRUENTRY_DOUBLE = 4;
+  private final static int LRUENTRY_NULL = 0;
+  private final static int LRUENTRY_STRING = 1;
+  private final static int LRUENTRY_INTEGER = 2;
+  private final static int LRUENTRY_LONG = 3;
+  private final static int LRUENTRY_DOUBLE = 4;
 
-  final static void assertLRUEntries(Set entries, int size, String keyPrefix, int instanceId) {
+  private static void assertLRUEntries(Set entries, int size, String keyPrefix, int instanceId) {
     assertEquals(size, entries.size());
     Iterator entItr = entries.iterator();
     while (entItr.hasNext()) {
@@ -5789,11 +5874,11 @@ public class TXJUnitTest {
   @Test
   public void testEviction() throws CacheException {
     final int lruSize = 8;
-    AttributesFactory af = new AttributesFactory();
+    AttributesFactory<String, Object> af = new AttributesFactory<>();
     af.setEvictionAttributes(
         EvictionAttributes.createLRUEntryAttributes(lruSize, EvictionAction.LOCAL_DESTROY));
     af.setScope(Scope.LOCAL);
-    Region lruRegion = this.cache.createRegion(getUniqueName(), af.create());
+    Region<String, Object> lruRegion = this.cache.createRegion(getUniqueName(), af.create());
 
     // Non-TX LRU verification
     assertEquals(0, lruRegion.entrySet(false).size());
@@ -5848,11 +5933,10 @@ public class TXJUnitTest {
     }
     clearRegion(lruRegion);
 
-
     // TX/non-TX no conflict verification w/ invalid initial state
     // full+2, all committed entries have TX refs using a loader
     {
-      AttributesMutator mutator = lruRegion.getAttributesMutator();
+      AttributesMutator<String, Object> mutator = lruRegion.getAttributesMutator();
       mutator.setCacheLoader(new CacheLoader() {
         // int count = 0;
         public Object load(LoaderHelper helper) throws CacheLoaderException {
@@ -5897,7 +5981,6 @@ public class TXJUnitTest {
     }
     clearRegion(lruRegion);
 
-
     // TX/TX/non-TX no conflict verification w/ initial state full, TX
     // add lruLimit+4, existing committed have TX 2 refs, force non-TX
     // eviction, force TX eviction
@@ -5931,11 +6014,6 @@ public class TXJUnitTest {
       tx2 = txMgrImpl.internalSuspend();
 
       assertLRUEntries(lruRegion.entrySet(false), lruSize, "key", LRUENTRY_INTEGER);
-      // LocalRegion lrReg = (LocalRegion) lruRegion;
-      // LRUClockNode lruE = null;
-      // assertNotNull(lruE = (LRUClockNode) lrReg.basicGetEntry("key"+(numToPut-1)));
-      // 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
@@ -6066,21 +6144,25 @@ public class TXJUnitTest {
 
     javax.transaction.TransactionManager jtaTxMgr = this.cache.getJTATransactionManager();
     TransactionListener tl = new TransactionListener() {
+      @Override
       public void afterCommit(TransactionEvent event) {
         ++listenerAfterCommit;
         te = event;
       }
 
+      @Override
       public void afterFailedCommit(TransactionEvent event) {
         ++listenerAfterFailedCommit;
         te = event;
       }
 
+      @Override
       public void afterRollback(TransactionEvent event) {
         ++listenerAfterRollback;
         te = event;
       }
 
+      @Override
       public void close() {
         ++listenerClose;
       }
@@ -6179,6 +6261,7 @@ public class TXJUnitTest {
       // a conflict
       final int signal[] = {0};
       Thread t = new Thread("non-TX conflict generator") {
+        @Override
         public void run() {
           try {
             region.put("syncKey4", "syncVal4");
@@ -6236,21 +6319,25 @@ public class TXJUnitTest {
       javax.transaction.HeuristicMixedException, javax.transaction.HeuristicRollbackException {
 
     TransactionListener tl = new TransactionListener() {
+      @Override
       public void afterCommit(TransactionEvent event) {
         ++listenerAfterCommit;
         te = event;
       }
 
+      @Override
       public void afterFailedCommit(TransactionEvent event) {
         ++listenerAfterFailedCommit;
         te = event;
       }
 
+      @Override
       public void afterRollback(TransactionEvent event) {
         ++listenerAfterRollback;
         te = event;
       }
 
+      @Override
       public void close() {
         ++listenerClose;
       }
@@ -6269,8 +6356,7 @@ public class TXJUnitTest {
       fail("Expected to get a healthy UserTransaction!");
     }
 
-
-    // Test enlistement for put
+    // Test enlistment for put
     // Test enlisted rollback
     // Test prevention of rollback/commit for enlisted transaction
     assertEquals(0, this.listenerAfterRollback);
@@ -6300,7 +6386,7 @@ public class TXJUnitTest {
     assertTrue(!this.region.containsKey("enlistKey"));
     assertEquals(1, this.listenerAfterRollback);
 
-    // Test enlistement for create
+    // Test enlistment for create
     // Test commit
     assertEquals(0, this.listenerAfterCommit);
     userTx.begin();
@@ -6313,7 +6399,7 @@ public class TXJUnitTest {
     assertEquals("enlistVal", this.region.getEntry("enlistKey").getValue());
     assertEquals(1, this.listenerAfterCommit);
 
-    // Test enlistement for get
+    // Test enlistment for get
     assertEquals(1, this.listenerAfterCommit);
     userTx.begin();
     assertEquals("enlistVal", this.region.get("enlistKey"));
@@ -6322,7 +6408,7 @@ public class TXJUnitTest {
     assertNull(this.txMgr.getTransactionId());
     assertEquals(2, this.listenerAfterCommit);
 
-    // Test enlistement for invalidate
+    // Test enlistment for invalidate
     assertEquals(2, this.listenerAfterCommit);
     userTx.begin();
     this.region.invalidate("enlistKey");
@@ -6335,7 +6421,7 @@ public class TXJUnitTest {
     assertTrue(!this.region.containsValueForKey("enlistKey"));
     assertEquals(3, this.listenerAfterCommit);
 
-    // Test enlistement for destroy
+    // Test enlistment for destroy
     assertEquals(3, this.listenerAfterCommit);
     userTx.begin();
     this.region.destroy("enlistKey");
@@ -6347,24 +6433,26 @@ public class TXJUnitTest {
     assertEquals(4, this.listenerAfterCommit);
 
     // Test enlistment for load
-    AttributesMutator mutator = this.region.getAttributesMutator();
-    mutator.setCacheLoader(new CacheLoader() {
+    AttributesMutator<String, String> mutator = this.region.getAttributesMutator();
+    mutator.setCacheLoader(new CacheLoader<String, String>() {
       int count = 0;
 
-      public Object load(LoaderHelper helper) throws CacheLoaderException {
-        return new Integer(count++);
+      @Override
+      public String load(LoaderHelper helper) throws CacheLoaderException {
+        return String.valueOf(count++);
       }
 
+      @Override
       public void close() {}
     });
     assertEquals(4, this.listenerAfterCommit);
     userTx.begin();
-    assertEquals(new Integer(0), this.region.get("enlistKey"));
+    assertEquals("0", this.region.get("enlistKey"));
     assertNotNull(this.txMgr.getTransactionId());
     userTx.commit();
     assertNull(this.txMgr.getTransactionId());
     assertTrue(this.region.containsKey("enlistKey"));
-    assertEquals(new Integer(0), this.region.getEntry("enlistKey").getValue());
+    assertEquals("0", this.region.getEntry("enlistKey").getValue());
     assertEquals(5, this.listenerAfterCommit);
     mutator.setCacheLoader(null);
 


[05/13] geode git commit: GEODE-2799: Handle different types of KeyInfo set when creating the KeySet Iterator.

Posted by kl...@apache.org.
GEODE-2799: Handle different types of KeyInfo set when creating the KeySet Iterator.


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

Branch: refs/heads/feature/GEODE-2632-6-1
Commit: 60ec931f53ead9a18950fb4b8e441ff3e9993820
Parents: 363e50d
Author: eshu <es...@pivotal.io>
Authored: Fri Apr 21 13:55:49 2017 -0700
Committer: eshu <es...@pivotal.io>
Committed: Fri Apr 21 13:55:49 2017 -0700

----------------------------------------------------------------------
 .../geode/internal/cache/LocalRegion.java       |  2 +-
 .../internal/cache/LocalRegionDataView.java     | 14 ++++-
 .../apache/geode/internal/cache/TXState.java    |  1 +
 .../cache/TXStateProxyImplJUnitTest.java        | 60 ++++++++++++++++++++
 4 files changed, 73 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/60ec931f/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index 5d5044b..45035d7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -11177,7 +11177,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
     return getRegionMap().keySet();
   }
 
-  public final InternalDataView getSharedDataView() {
+  public InternalDataView getSharedDataView() {
     return this.sharedDataView;
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/60ec931f/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegionDataView.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegionDataView.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegionDataView.java
index b4aa20b..8db979b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegionDataView.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegionDataView.java
@@ -200,11 +200,19 @@ public class LocalRegionDataView implements InternalDataView {
    */
   public Object getKeyForIterator(final KeyInfo keyInfo, final LocalRegion currRgn,
       boolean rememberReads, boolean allowTombstones) {
-    final AbstractRegionEntry re = (AbstractRegionEntry) keyInfo.getKey();
+    final Object key = keyInfo.getKey();
+    if (key == null) {
+      return null;
+    }
     // fix for 42182, before returning a key verify that its value
     // is not a removed token
-    if (re != null && (!re.isDestroyedOrRemoved() || (allowTombstones && re.isTombstone()))) {
-      return re.getKey();
+    if (key instanceof RegionEntry) {
+      RegionEntry re = (RegionEntry) key;
+      if (!re.isDestroyedOrRemoved() || (allowTombstones && re.isTombstone())) {
+        return re.getKey();
+      }
+    } else if (getEntry(keyInfo, currRgn, allowTombstones) != null) {
+      return key;
     }
     return null;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/60ec931f/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java b/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
index 234baee..0a9d80e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/TXState.java
@@ -1705,6 +1705,7 @@ public class TXState implements TXStateInterface {
    */
   public Object getKeyForIterator(KeyInfo curr, LocalRegion currRgn, boolean rememberReads,
       boolean allowTombstones) {
+    assert !(curr.getKey() instanceof RegionEntry);
     if (!readEntryAndCheckIfDestroyed(curr, currRgn, rememberReads)) {
       return curr.getKey();
     } else {

http://git-wip-us.apache.org/repos/asf/geode/blob/60ec931f/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateProxyImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateProxyImplJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateProxyImplJUnitTest.java
new file mode 100644
index 0000000..9ce76c5
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/TXStateProxyImplJUnitTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.internal.cache;
+
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.LocalRegion.NonTXEntry;
+import org.apache.geode.internal.cache.region.entry.RegionEntryFactoryBuilder;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class TXStateProxyImplJUnitTest {
+  @Test
+  public void testGetKeyForIterator() {
+    RegionEntryFactory factory = new RegionEntryFactoryBuilder().getRegionEntryFactoryOrNull(false,
+        false, false, false, false);
+    LocalRegion region = mock(LocalRegion.class);
+    String key = "testkey";
+    RegionEntry re = factory.createEntry(region, key, null);
+    TXId txId = new TXId(mock(InternalDistributedMember.class), 1);
+    TXStateProxyImpl tx = new TXStateProxyImpl(mock(TXManagerImpl.class), txId, false);
+    LocalRegionDataView view = mock(LocalRegionDataView.class);
+    boolean allowTombstones = false;
+    boolean rememberReads = true;
+
+    KeyInfo stringKeyInfo = new KeyInfo(key, null, null);
+    KeyInfo regionEntryKeyInfo = new KeyInfo(re, null, null);
+
+    when(region.getSharedDataView()).thenReturn(view);
+    when(view.getEntry(stringKeyInfo, region, allowTombstones)).thenReturn(mock(NonTXEntry.class));
+    when(view.getKeyForIterator(stringKeyInfo, region, rememberReads, allowTombstones))
+        .thenCallRealMethod();
+    when(view.getKeyForIterator(regionEntryKeyInfo, region, rememberReads, allowTombstones))
+        .thenCallRealMethod();
+
+    Object key1 = tx.getKeyForIterator(regionEntryKeyInfo, region, rememberReads, allowTombstones);
+    assertTrue(key1.equals(key));
+    Object key2 = tx.getKeyForIterator(stringKeyInfo, region, rememberReads, allowTombstones);
+    assertTrue(key2.equals(key));
+  }
+
+}


[07/13] geode git commit: GEODE-2808 - Fixing lock ordering issues in DeltaSession

Posted by kl...@apache.org.
GEODE-2808 - Fixing lock ordering issues in DeltaSession

Region expiration of sessions and explicit expiration of sessions had
lock ordering issues. Fixing the code so that expiration goes through
the region entry lock first, before getting the lock on StandardSession.

Adding a workaround for the fact that liferay calls removeAttribute
from within session expiration by ignoreing remoteAttribute calls during
expiration.

This closes #472


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

Branch: refs/heads/feature/GEODE-2632-6-1
Commit: 45dc6744154a08986e833852ef743af5d8bf19ba
Parents: 47d8c82
Author: Dan Smith <up...@apache.org>
Authored: Fri Apr 21 11:36:24 2017 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri Apr 21 16:23:24 2017 -0700

----------------------------------------------------------------------
 .../modules/session/catalina/DeltaSession7.java | 14 +++++++-
 .../modules/session/catalina/DeltaSession8.java | 14 +++++++-
 .../session/TestSessionsTomcat8Base.java        | 34 ++++++++++++++++++++
 .../modules/session/catalina/DeltaSession.java  | 14 +++++++-
 .../geode/modules/session/CommandServlet.java   |  4 +++
 .../geode/modules/session/QueryCommand.java     |  2 ++
 .../geode/modules/session/TestSessionsBase.java | 34 ++++++++++++++++++++
 7 files changed, 113 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/45dc6744/extensions/geode-modules-tomcat7/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession7.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-tomcat7/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession7.java b/extensions/geode-modules-tomcat7/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession7.java
index 204ff5e..d7f30bd 100644
--- a/extensions/geode-modules-tomcat7/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession7.java
+++ b/extensions/geode-modules-tomcat7/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession7.java
@@ -263,6 +263,9 @@ public class DeltaSession7 extends StandardSession
 
   public void removeAttribute(String name, boolean notify) {
     checkBackingCacheAvailable();
+    if (expired) {
+      return;
+    }
     synchronized (this.changeLock) {
       // Remove the attribute locally
       super.removeAttribute(name, true);
@@ -322,7 +325,7 @@ public class DeltaSession7 extends StandardSession
     setExpired(true);
 
     // Do expire processing
-    expire();
+    super.expire(true);
 
     // Update statistics
     if (manager != null) {
@@ -330,6 +333,15 @@ public class DeltaSession7 extends StandardSession
     }
   }
 
+  @Override
+  public void expire(boolean notify) {
+    if (notify) {
+      getOperatingRegion().destroy(this.getId(), this);
+    } else {
+      super.expire(false);
+    }
+  }
+
   public void setMaxInactiveInterval(int interval) {
     super.setMaxInactiveInterval(interval);
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/45dc6744/extensions/geode-modules-tomcat8/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession8.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-tomcat8/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession8.java b/extensions/geode-modules-tomcat8/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession8.java
index b5e7d0c..f69382a 100644
--- a/extensions/geode-modules-tomcat8/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession8.java
+++ b/extensions/geode-modules-tomcat8/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession8.java
@@ -258,6 +258,9 @@ public class DeltaSession8 extends StandardSession
 
   public void removeAttribute(String name, boolean notify) {
     checkBackingCacheAvailable();
+    if (expired) {
+      return;
+    }
     synchronized (this.changeLock) {
       // Remove the attribute locally
       super.removeAttribute(name, true);
@@ -317,7 +320,7 @@ public class DeltaSession8 extends StandardSession
     setExpired(true);
 
     // Do expire processing
-    expire();
+    super.expire(true);
 
     // Update statistics
     if (manager != null) {
@@ -325,6 +328,15 @@ public class DeltaSession8 extends StandardSession
     }
   }
 
+  @Override
+  public void expire(boolean notify) {
+    if (notify) {
+      getOperatingRegion().destroy(this.getId(), this);
+    } else {
+      super.expire(false);
+    }
+  }
+
   public void setMaxInactiveInterval(int interval) {
     super.setMaxInactiveInterval(interval);
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/45dc6744/extensions/geode-modules-tomcat8/src/test/java/org/apache/geode/modules/session/TestSessionsTomcat8Base.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-tomcat8/src/test/java/org/apache/geode/modules/session/TestSessionsTomcat8Base.java b/extensions/geode-modules-tomcat8/src/test/java/org/apache/geode/modules/session/TestSessionsTomcat8Base.java
index 15b3874..1dc1d8b 100644
--- a/extensions/geode-modules-tomcat8/src/test/java/org/apache/geode/modules/session/TestSessionsTomcat8Base.java
+++ b/extensions/geode-modules-tomcat8/src/test/java/org/apache/geode/modules/session/TestSessionsTomcat8Base.java
@@ -233,6 +233,40 @@ public abstract class TestSessionsTomcat8Base extends JUnit4DistributedTestCase
   }
 
   /**
+   * Test expiration of a session by the tomcat container, rather than gemfire expiration
+   */
+  @Test
+  public void testSessionExpirationByContainer() throws Exception {
+
+    String key = "value_testSessionExpiration1";
+    String value = "Foo";
+
+    WebConversation wc = new WebConversation();
+    WebRequest req = new GetMethodWebRequest(String.format("http://localhost:%d/test", port));
+
+    // Set an attribute
+    req.setParameter("cmd", QueryCommand.SET.name());
+    req.setParameter("param", key);
+    req.setParameter("value", value);
+    WebResponse response = wc.getResponse(req);
+
+    // Set the session timeout of this one session.
+    req.setParameter("cmd", QueryCommand.SET_MAX_INACTIVE.name());
+    req.setParameter("value", "1");
+    response = wc.getResponse(req);
+
+    // Wait until the session should expire
+    Thread.sleep(2000);
+
+    // Do a request, which should cause the session to be expired
+    req.setParameter("cmd", QueryCommand.GET.name());
+    req.setParameter("param", key);
+    response = wc.getResponse(req);
+
+    assertEquals("", response.getText());
+  }
+
+  /**
    * Test that removing a session attribute also removes it from the region
    */
   @Test

http://git-wip-us.apache.org/repos/asf/geode/blob/45dc6744/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession.java b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession.java
index bc421a5..ac612da 100644
--- a/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession.java
+++ b/extensions/geode-modules/src/main/java/org/apache/geode/modules/session/catalina/DeltaSession.java
@@ -266,6 +266,9 @@ public class DeltaSession extends StandardSession
 
   public void removeAttribute(String name, boolean notify) {
     checkBackingCacheAvailable();
+    if (expired) {
+      return;
+    }
     synchronized (this.changeLock) {
       // Remove the attribute locally
       super.removeAttribute(name, true);
@@ -325,7 +328,7 @@ public class DeltaSession extends StandardSession
     setExpired(true);
 
     // Do expire processing
-    expire();
+    super.expire(true);
 
     // Update statistics
     if (manager != null) {
@@ -333,6 +336,15 @@ public class DeltaSession extends StandardSession
     }
   }
 
+  @Override
+  public void expire(boolean notify) {
+    if (notify) {
+      getOperatingRegion().destroy(this.getId(), this);
+    } else {
+      super.expire(false);
+    }
+  }
+
   public void setMaxInactiveInterval(int interval) {
     super.setMaxInactiveInterval(interval);
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/45dc6744/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/CommandServlet.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/CommandServlet.java b/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/CommandServlet.java
index 3fede62..a04194b 100644
--- a/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/CommandServlet.java
+++ b/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/CommandServlet.java
@@ -60,6 +60,10 @@ public class CommandServlet extends HttpServlet {
         session = request.getSession();
         session.setAttribute(param, value);
         break;
+      case SET_MAX_INACTIVE:
+        session = request.getSession();
+        session.setMaxInactiveInterval(Integer.valueOf(value));
+        break;
       case GET:
         session = request.getSession();
         String val = (String) session.getAttribute(param);

http://git-wip-us.apache.org/repos/asf/geode/blob/45dc6744/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/QueryCommand.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/QueryCommand.java b/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/QueryCommand.java
index 2b89e68..622866e 100644
--- a/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/QueryCommand.java
+++ b/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/QueryCommand.java
@@ -21,6 +21,8 @@ public enum QueryCommand {
 
   SET,
 
+  SET_MAX_INACTIVE,
+
   GET,
 
   INVALIDATE,

http://git-wip-us.apache.org/repos/asf/geode/blob/45dc6744/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/TestSessionsBase.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/TestSessionsBase.java b/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/TestSessionsBase.java
index d7674dd..a6bec6c 100644
--- a/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/TestSessionsBase.java
+++ b/extensions/geode-modules/src/test/java/org/apache/geode/modules/session/TestSessionsBase.java
@@ -267,6 +267,40 @@ public abstract class TestSessionsBase {
   }
 
   /**
+   * Test expiration of a session by the tomcat container, rather than gemfire expiration
+   */
+  @Test
+  public void testSessionExpirationByContainer() throws Exception {
+
+    String key = "value_testSessionExpiration1";
+    String value = "Foo";
+
+    WebConversation wc = new WebConversation();
+    WebRequest req = new GetMethodWebRequest(String.format("http://localhost:%d/test", port));
+
+    // Set an attribute
+    req.setParameter("cmd", QueryCommand.SET.name());
+    req.setParameter("param", key);
+    req.setParameter("value", value);
+    WebResponse response = wc.getResponse(req);
+
+    // Set the session timeout of this one session.
+    req.setParameter("cmd", QueryCommand.SET_MAX_INACTIVE.name());
+    req.setParameter("value", "1");
+    response = wc.getResponse(req);
+
+    // Wait until the session should expire
+    Thread.sleep(2000);
+
+    // Do a request, which should cause the session to be expired
+    req.setParameter("cmd", QueryCommand.GET.name());
+    req.setParameter("param", key);
+    response = wc.getResponse(req);
+
+    assertEquals("", response.getText());
+  }
+
+  /**
    * Test that removing a session attribute also removes it from the region
    */
   @Test


[04/13] geode git commit: GEODE-2632: refactor code to use InternalCache instead of GemFireCacheImpl

Posted by kl...@apache.org.
GEODE-2632: refactor code to use InternalCache instead of GemFireCacheImpl

* minor cleanup also


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

Branch: refs/heads/feature/GEODE-2632-6-1
Commit: 363e50d213d763f4cca6e0744b206941a4f2d52c
Parents: 0862174
Author: Kirk Lund <kl...@apache.org>
Authored: Wed Apr 19 14:41:42 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Apr 21 13:45:22 2017 -0700

----------------------------------------------------------------------
 .../query/internal/cq/CqServiceProvider.java    |  22 +-
 .../query/internal/cq/spi/CqServiceFactory.java |   8 +-
 .../cache/query/internal/cq/ClientCQImpl.java   |  95 +--
 .../cache/query/internal/cq/CqQueryImpl.java    |  91 ++-
 .../query/internal/cq/CqServiceFactoryImpl.java |  17 +-
 .../cache/query/internal/cq/CqServiceImpl.java  | 673 ++++---------------
 .../internal/cq/CqServiceStatisticsImpl.java    |  21 +-
 .../query/internal/cq/CqServiceVsdStats.java    |  73 +-
 .../cache/query/internal/cq/ServerCQImpl.java   | 121 +---
 .../cache/tier/sockets/command/ExecuteCQ.java   |   4 +-
 .../cache/tier/sockets/command/ExecuteCQ61.java |   4 +-
 .../tier/sockets/command/GetDurableCQs.java     |   4 +-
 .../cache/query/cq/dunit/CqStatsDUnitTest.java  |  44 +-
 .../cq/dunit/CqStatsUsingPoolDUnitTest.java     |  47 +-
 .../TopEntriesFunctionCollector.java            |  22 +-
 .../LuceneQueryFunctionJUnitTest.java           |  54 +-
 .../TopEntriesCollectorJUnitTest.java           |  23 +-
 .../TopEntriesFunctionCollectorJUnitTest.java   |  48 +-
 .../distributed/TopEntriesJUnitTest.java        |  44 +-
 .../cache/lucene/test/LuceneTestUtilities.java  |  16 +
 20 files changed, 442 insertions(+), 989 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceProvider.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceProvider.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceProvider.java
index cded9c3..90fbf4b 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceProvider.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceProvider.java
@@ -16,7 +16,7 @@ package org.apache.geode.cache.query.internal.cq;
 
 import org.apache.geode.cache.query.internal.cq.spi.CqServiceFactory;
 import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 
 import java.io.DataInput;
 import java.io.IOException;
@@ -26,17 +26,19 @@ import java.util.ServiceLoader;
 public class CqServiceProvider {
 
   private static final CqServiceFactory factory;
-  // System property to maintain the CQ event references for optimizing the updates.
-  // This will allows to run the CQ query only once during update events.
+
+  /**
+   * System property to maintain the CQ event references for optimizing the updates. This will allow
+   * running the CQ query only once during update events.
+   */
   public static boolean MAINTAIN_KEYS = Boolean
-      .valueOf(System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "cq.MAINTAIN_KEYS", "true"))
-      .booleanValue();
+      .valueOf(System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "cq.MAINTAIN_KEYS", "true"));
+
   /**
    * A debug flag used for testing vMotion during CQ registration
    */
   public static boolean VMOTION_DURING_CQ_REGISTRATION_FLAG = false;
 
-
   static {
     ServiceLoader<CqServiceFactory> loader = ServiceLoader.load(CqServiceFactory.class);
     Iterator<CqServiceFactory> itr = loader.iterator();
@@ -48,8 +50,7 @@ public class CqServiceProvider {
     }
   }
 
-  public static CqService create(GemFireCacheImpl cache) {
-
+  public static CqService create(InternalCache cache) {
     if (factory == null) {
       return new MissingCqService();
     }
@@ -63,10 +64,7 @@ public class CqServiceProvider {
     } else {
       return factory.readCqQuery(in);
     }
-
   }
 
-  private CqServiceProvider() {
-
-  }
+  private CqServiceProvider() {}
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/spi/CqServiceFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/spi/CqServiceFactory.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/spi/CqServiceFactory.java
index 68ebbd5..2b8a47e 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/spi/CqServiceFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/cq/spi/CqServiceFactory.java
@@ -19,16 +19,16 @@ import java.io.IOException;
 
 import org.apache.geode.cache.query.internal.cq.CqService;
 import org.apache.geode.cache.query.internal.cq.ServerCQ;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 
 public interface CqServiceFactory {
 
-  public void initialize();
+  void initialize();
 
   /**
    * Create a new CqService for the given cache
    */
-  public CqService create(GemFireCacheImpl cache);
+  CqService create(InternalCache cache);
 
-  public ServerCQ readCqQuery(DataInput in) throws ClassNotFoundException, IOException;
+  ServerCQ readCqQuery(DataInput in) throws ClassNotFoundException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ClientCQImpl.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ClientCQImpl.java b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ClientCQImpl.java
index 00a0aa5..111bf84 100644
--- a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ClientCQImpl.java
+++ b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ClientCQImpl.java
@@ -35,7 +35,7 @@ import org.apache.geode.cache.query.CqResults;
 import org.apache.geode.cache.query.CqStatusListener;
 import org.apache.geode.cache.query.RegionNotFoundException;
 import org.apache.geode.cache.query.internal.CqStateImpl;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
@@ -57,7 +57,7 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
    */
   private volatile ConcurrentLinkedQueue<CqEventImpl> queuedEvents = null;
 
-  public final Object queuedEventsSynchObject = new Object();
+  final Object queuedEventsSynchObject = new Object();
 
   private boolean connected = false;
 
@@ -73,22 +73,15 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
     return this.cqName;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqQuery2#getCQProxy()
-   */
-  public ServerCQProxyImpl getCQProxy() {
+  ServerCQProxyImpl getCQProxy() {
     return this.cqProxy;
   }
 
   /**
    * Initializes the connection using the pool from the client region. Also sets the cqBaseRegion
    * value of this CQ.
-   * 
-   * @throws CqException
    */
-  public void initConnectionProxy() throws CqException, RegionNotFoundException {
+  private void initConnectionProxy() throws CqException, RegionNotFoundException {
     cqBaseRegion = (LocalRegion) cqService.getCache().getRegion(regionName);
     // Check if the region exists on the local cache.
     // In the current implementation of 5.1 the Server Connection is (ConnectionProxyImpl)
@@ -113,17 +106,9 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
       throw new CqException(
           "Unable to get the connection pool. The Region does not have a pool configured.");
     }
-
-    // if (proxy == null) {
-    // throw new
-    // CqException(LocalizedStrings.CqQueryImpl_UNABLE_TO_GET_THE_CONNECTIONPROXY_THE_REGION_MAY_NOT_HAVE_A_BRIDGEWRITER_OR_BRIDGECLIENT_INSTALLED_ON_IT.toLocalizedString());
-    // } else if(!proxy.getEstablishCallbackConnection()){
-    // throw new
-    // CqException(LocalizedStrings.CqQueryImpl_THE_ESTABLISHCALLBACKCONNECTION_ON_BRIDGEWRITER_CLIENT_INSTALLED_ON_REGION_0_IS_SET_TO_FALSE
-    // .toLocalizedString(regionName));
-    // }
   }
 
+  @Override
   public void close() throws CqClosedException, CqException {
     this.close(true);
   }
@@ -182,15 +167,15 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
       if (cqProxy == null || !sendRequestToServer || isClosed) {
         // Stat update.
         if (stateBeforeClosing == CqStateImpl.RUNNING) {
-          cqService.stats.decCqsActive();
+          cqService.stats().decCqsActive();
         } else if (stateBeforeClosing == CqStateImpl.STOPPED) {
-          cqService.stats.decCqsStopped();
+          cqService.stats().decCqsStopped();
         }
 
         // Set the state to close, and update stats
         this.cqState.setState(CqStateImpl.CLOSED);
-        cqService.stats.incCqsClosed();
-        cqService.stats.decCqsOnClient();
+        cqService.stats().incCqsClosed();
+        cqService.stats().decCqsOnClient();
         if (this.stats != null)
           this.stats.close();
       } else {
@@ -201,7 +186,7 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
         if (exception != null) {
           throw new CqException(
               LocalizedStrings.CqQueryImpl_FAILED_TO_CLOSE_THE_CQ_CQNAME_0_ERROR_FROM_LAST_ENDPOINT_1
-                  .toLocalizedString(new Object[] {this.cqName, exception.getLocalizedMessage()}),
+                  .toLocalizedString(this.cqName, exception.getLocalizedMessage()),
               exception.getCause());
         } else {
           throw new CqException(
@@ -261,31 +246,28 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
 
   /**
    * Clears the resource used by CQ.
-   * 
-   * @throws CqException
    */
+  @Override
   protected void cleanup() throws CqException {
     this.cqService.removeFromBaseRegionToCqNameMap(this.regionName, this.getServerCqName());
   }
 
+  @Override
   public CqAttributes getCqAttributes() {
     return cqAttributes;
   }
 
-
-
   /**
    * @return Returns the cqListeners.
    */
   public CqListener[] getCqListeners() {
-
     return cqAttributes.getCqListeners();
   }
 
-
   /**
    * Start or resume executing the query.
    */
+  @Override
   public void execute() throws CqClosedException, RegionNotFoundException, CqException {
     executeCqOnRedundantsAndPrimary(false);
   }
@@ -293,7 +275,8 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
   /**
    * Start or resume executing the query. Gets or updates the CQ results and returns them.
    */
-  public CqResults executeWithInitialResults()
+  @Override
+  public <E> CqResults<E> executeWithInitialResults()
       throws CqClosedException, RegionNotFoundException, CqException {
 
     synchronized (queuedEventsSynchObject) {
@@ -320,16 +303,7 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
     CqResults initialResults;
     try {
       initialResults = (CqResults) executeCqOnRedundantsAndPrimary(true);
-    } catch (CqClosedException e) {
-      queuedEvents = null;
-      throw e;
-    } catch (RegionNotFoundException e) {
-      queuedEvents = null;
-      throw e;
-    } catch (CqException e) {
-      queuedEvents = null;
-      throw e;
-    } catch (RuntimeException e) {
+    } catch (RegionNotFoundException | CqException | RuntimeException e) {
       queuedEvents = null;
       throw e;
     }
@@ -343,6 +317,7 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
         if (!this.queuedEvents.isEmpty()) {
           try {
             Runnable r = new Runnable() {
+              @Override
               public void run() {
                 Object[] eventArray = null;
                 if (CqQueryImpl.testHook != null) {
@@ -395,7 +370,7 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
    * @param executeWithInitialResults boolean
    * @return Object SelectResults in case of executeWithInitialResults
    */
-  public Object executeCqOnRedundantsAndPrimary(boolean executeWithInitialResults)
+  private Object executeCqOnRedundantsAndPrimary(boolean executeWithInitialResults)
       throws CqClosedException, RegionNotFoundException, CqException {
 
     Object initialResults = null;
@@ -461,8 +436,7 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
           } else {
             String errMsg =
                 LocalizedStrings.CqQueryImpl_FAILED_TO_EXECUTE_THE_CQ_CQNAME_0_QUERY_STRING_IS_1_ERROR_FROM_LAST_SERVER_2
-                    .toLocalizedString(
-                        new Object[] {this.cqName, this.queryString, ex.getLocalizedMessage()});
+                    .toLocalizedString(this.cqName, this.queryString, ex.getLocalizedMessage());
             if (logger.isDebugEnabled()) {
               logger.debug(errMsg, ex);
             }
@@ -498,8 +472,8 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
       }
     }
     // Update CQ-base region for book keeping.
-    this.cqService.stats.incCqsActive();
-    this.cqService.stats.decCqsStopped();
+    this.cqService.stats().incCqsActive();
+    this.cqService.stats().decCqsStopped();
     return initialResults;
   }
 
@@ -509,23 +483,22 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
    * @return true if shutdown in progress else false.
    */
   private boolean shutdownInProgress() {
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+    InternalCache cache = cqService.getInternalCache();
     if (cache == null || cache.isClosed()) {
       return true; // bail, things are shutting down
     }
 
-
     String reason = cqProxy.getPool().getCancelCriterion().cancelInProgress();
     if (reason != null) {
       return true;
     }
     return false;
-
   }
 
   /**
    * Stop or pause executing the query.
    */
+  @Override
   public void stop() throws CqClosedException, CqException {
     boolean isStopped = false;
     synchronized (this.cqState) {
@@ -558,8 +531,8 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
       if (cqProxy == null || isStopped) {
         // Change state and stats on the client side
         this.cqState.setState(CqStateImpl.STOPPED);
-        this.cqService.stats.incCqsStopped();
-        this.cqService.stats.decCqsActive();
+        this.cqService.stats().incCqsStopped();
+        this.cqService.stats().decCqsActive();
         if (logger.isDebugEnabled()) {
           logger.debug("Successfully stopped the CQ. {}", cqName);
         }
@@ -568,7 +541,7 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
         if (exception != null) {
           throw new CqException(
               LocalizedStrings.CqQueryImpl_FAILED_TO_STOP_THE_CQ_CQNAME_0_ERROR_FROM_LAST_SERVER_1
-                  .toLocalizedString(new Object[] {this.cqName, exception.getLocalizedMessage()}),
+                  .toLocalizedString(this.cqName, exception.getLocalizedMessage()),
               exception.getCause());
         } else {
           throw new CqException(
@@ -579,24 +552,15 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
     }
   }
 
+  @Override
   public CqAttributesMutator getCqAttributesMutator() {
     return (CqAttributesMutator) this.cqAttributes;
   }
 
-
-  public ConcurrentLinkedQueue<CqEventImpl> getQueuedEvents() {
+  ConcurrentLinkedQueue<CqEventImpl> getQueuedEvents() {
     return this.queuedEvents;
   }
 
-
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqQuery2#setProxyCache(org.apache.geode.cache.
-   * client.internal.ProxyCache)
-   */
   @Override
   public void setProxyCache(ProxyCache proxyCache) {
     this.proxyCache = proxyCache;
@@ -612,7 +576,6 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
 
   @Override
   public void createOn(Connection conn, boolean isDurable) {
-
     byte regionDataPolicyOrdinal = getCqBaseRegion() == null ? (byte) 0
         : getCqBaseRegion().getAttributes().getDataPolicy().ordinal;
 
@@ -620,6 +583,4 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
     this.cqProxy.createOn(getName(), conn, getQueryString(), state, isDurable,
         regionDataPolicyOrdinal);
   }
-
-
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqQueryImpl.java b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqQueryImpl.java
index 22b4137..07e3171 100644
--- a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqQueryImpl.java
+++ b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqQueryImpl.java
@@ -21,11 +21,9 @@ import java.util.Set;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.StatisticsFactory;
-import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.query.CqClosedException;
 import org.apache.geode.cache.query.CqEvent;
 import org.apache.geode.cache.query.CqException;
-import org.apache.geode.cache.query.CqExistsException;
 import org.apache.geode.cache.query.CqState;
 import org.apache.geode.cache.query.CqStatistics;
 import org.apache.geode.cache.query.Query;
@@ -38,7 +36,7 @@ import org.apache.geode.cache.query.internal.CqStateImpl;
 import org.apache.geode.cache.query.internal.DefaultQuery;
 import org.apache.geode.cache.query.internal.ExecutionContext;
 import org.apache.geode.cache.query.internal.QueryExecutionContext;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.InternalLogWriter;
@@ -58,13 +56,13 @@ public abstract class CqQueryImpl implements InternalCqQuery {
 
   protected String queryString;
 
-  protected static final Object TOKEN = new Object();
+  static final Object TOKEN = new Object();
 
-  protected LocalRegion cqBaseRegion;
+  LocalRegion cqBaseRegion;
 
   protected Query query = null;
 
-  protected InternalLogWriter securityLogWriter;
+  InternalLogWriter securityLogWriter;
 
   protected CqServiceImpl cqService;
 
@@ -72,14 +70,14 @@ public abstract class CqQueryImpl implements InternalCqQuery {
 
   protected boolean isDurable = false;
 
-  // Stats counters
-  protected CqStatisticsImpl cqStats;
+  /** Stats counters */
+  private CqStatisticsImpl cqStats;
 
   protected CqQueryVsdStats stats;
 
   protected final CqStateImpl cqState = new CqStateImpl();
 
-  protected ExecutionContext queryExecutionContext = null;
+  private ExecutionContext queryExecutionContext = null;
 
   public static TestHook testHook = null;
 
@@ -100,6 +98,7 @@ public abstract class CqQueryImpl implements InternalCqQuery {
   /**
    * returns CQ name
    */
+  @Override
   public String getName() {
     return this.cqName;
   }
@@ -109,6 +108,7 @@ public abstract class CqQueryImpl implements InternalCqQuery {
     this.cqName = cqName;
   }
 
+  @Override
   public void setCqService(CqService cqService) {
     this.cqService = (CqServiceImpl) cqService;
   }
@@ -121,25 +121,24 @@ public abstract class CqQueryImpl implements InternalCqQuery {
     return this.regionName;
   }
 
-  public void updateCqCreateStats() {
+  void updateCqCreateStats() {
     // Initialize the VSD statistics
     StatisticsFactory factory = cqService.getCache().getDistributedSystem();
     this.stats = new CqQueryVsdStats(factory, getServerCqName());
     this.cqStats = new CqStatisticsImpl(this);
 
     // Update statistics with CQ creation.
-    this.cqService.stats.incCqsStopped();
-    this.cqService.stats.incCqsCreated();
-    this.cqService.stats.incCqsOnClient();
+    this.cqService.stats().incCqsStopped();
+    this.cqService.stats().incCqsCreated();
+    this.cqService.stats().incCqsOnClient();
   }
 
   /**
    * Validates the CQ. Checks for cq constraints. Also sets the base region name.
    */
-  public void validateCq() {
-    Cache cache = cqService.getCache();
-    DefaultQuery locQuery =
-        (DefaultQuery) ((GemFireCacheImpl) cache).getLocalQueryService().newQuery(this.queryString);
+  void validateCq() {
+    InternalCache cache = cqService.getInternalCache();
+    DefaultQuery locQuery = (DefaultQuery) cache.getLocalQueryService().newQuery(this.queryString);
     this.query = locQuery;
     // assert locQuery != null;
 
@@ -221,10 +220,8 @@ public abstract class CqQueryImpl implements InternalCqQuery {
 
   /**
    * Removes the CQ from CQ repository.
-   * 
-   * @throws CqException
    */
-  protected void removeFromCqMap() throws CqException {
+  void removeFromCqMap() throws CqException {
     try {
       cqService.removeCq(this.getServerCqName());
     } catch (Exception ex) {
@@ -243,6 +240,7 @@ public abstract class CqQueryImpl implements InternalCqQuery {
   /**
    * Returns the QueryString of this CQ.
    */
+  @Override
   public String getQueryString() {
     return queryString;
   }
@@ -252,23 +250,16 @@ public abstract class CqQueryImpl implements InternalCqQuery {
    * 
    * @return the Query for the query string
    */
+  @Override
   public Query getQuery() {
     return query;
   }
 
-
-  /**
-   * @see org.apache.geode.cache.query.CqQuery#getStatistics()
-   */
+  @Override
   public CqStatistics getStatistics() {
     return cqStats;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqQuery2#getCqBaseRegion()
-   */
   @Override
   public LocalRegion getCqBaseRegion() {
     return this.cqBaseRegion;
@@ -279,11 +270,12 @@ public abstract class CqQueryImpl implements InternalCqQuery {
   /**
    * @return Returns the Region name on which this cq is created.
    */
-  public String getBaseRegionName() {
+  String getBaseRegionName() {
 
     return this.regionName;
   }
 
+  @Override
   public abstract String getServerCqName();
 
   /**
@@ -291,15 +283,11 @@ public abstract class CqQueryImpl implements InternalCqQuery {
    * 
    * @return STOPPED RUNNING or CLOSED
    */
+  @Override
   public CqState getState() {
     return this.cqState;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqQuery2#setCqState(int)
-   */
   @Override
   public void setCqState(int state) {
     if (this.isClosed()) {
@@ -309,18 +297,13 @@ public abstract class CqQueryImpl implements InternalCqQuery {
 
     synchronized (cqState) {
       if (state == CqStateImpl.RUNNING) {
-        if (this.isRunning()) {
-          // throw new
-          // IllegalStateException(LocalizedStrings.CqQueryImpl_CQ_IS_NOT_IN_RUNNING_STATE_STOP_CQ_DOES_NOT_APPLY_CQNAME_0
-          // .toLocalizedString(this.cqName));
-        }
         this.cqState.setState(CqStateImpl.RUNNING);
-        this.cqService.stats.incCqsActive();
-        this.cqService.stats.decCqsStopped();
+        this.cqService.stats().incCqsActive();
+        this.cqService.stats().decCqsStopped();
       } else if (state == CqStateImpl.STOPPED) {
         this.cqState.setState(CqStateImpl.STOPPED);
-        this.cqService.stats.incCqsStopped();
-        this.cqService.stats.decCqsActive();
+        this.cqService.stats().incCqsStopped();
+        this.cqService.stats().decCqsActive();
       } else if (state == CqStateImpl.CLOSING) {
         this.cqState.setState(state);
       }
@@ -332,7 +315,7 @@ public abstract class CqQueryImpl implements InternalCqQuery {
    * 
    * @param cqEvent object
    */
-  public void updateStats(CqEvent cqEvent) {
+  void updateStats(CqEvent cqEvent) {
     this.stats.updateStats(cqEvent); // Stats for VSD
   }
 
@@ -341,15 +324,17 @@ public abstract class CqQueryImpl implements InternalCqQuery {
    * 
    * @return true if running, false otherwise
    */
+  @Override
   public boolean isRunning() {
     return this.cqState.isRunning();
   }
 
   /**
-   * Return true if the CQ is in Sstopped state
+   * Return true if the CQ is in stopped state
    * 
    * @return true if stopped, false otherwise
    */
+  @Override
   public boolean isStopped() {
     return this.cqState.isStopped();
   }
@@ -359,6 +344,7 @@ public abstract class CqQueryImpl implements InternalCqQuery {
    * 
    * @return true if closed, false otherwise
    */
+  @Override
   public boolean isClosed() {
     return this.cqState.isClosed();
   }
@@ -377,6 +363,7 @@ public abstract class CqQueryImpl implements InternalCqQuery {
    * 
    * @return true if durable, false otherwise
    */
+  @Override
   public boolean isDurable() {
     return this.isDurable;
   }
@@ -391,22 +378,22 @@ public abstract class CqQueryImpl implements InternalCqQuery {
     return stats;
   }
 
-  public ExecutionContext getQueryExecutionContext() {
+  ExecutionContext getQueryExecutionContext() {
     return queryExecutionContext;
   }
 
-  public void setQueryExecutionContext(ExecutionContext queryExecutionContext) {
+  private void setQueryExecutionContext(ExecutionContext queryExecutionContext) {
     this.queryExecutionContext = queryExecutionContext;
   }
 
   /** Test Hook */
   public interface TestHook {
-    public void pauseUntilReady();
+    void pauseUntilReady();
 
-    public void ready();
+    void ready();
 
-    public int numQueuedEvents();
+    int numQueuedEvents();
 
-    public void setEventCount(int count);
+    void setEventCount(int count);
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceFactoryImpl.java b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceFactoryImpl.java
index db90632..9cc2eea 100644
--- a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceFactoryImpl.java
+++ b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceFactoryImpl.java
@@ -22,7 +22,7 @@ import java.util.Map;
 
 import org.apache.geode.cache.query.internal.cq.spi.CqServiceFactory;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.tier.Command;
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.CommandInitializer;
@@ -36,14 +36,13 @@ import org.apache.geode.internal.cache.tier.sockets.command.StopCQ;
 
 public class CqServiceFactoryImpl implements CqServiceFactory {
 
+  @Override
   public void initialize() {
-    {
-      Map<Version, Command> versions = new HashMap<Version, Command>();
-      versions.put(Version.GFE_57, ExecuteCQ.getCommand());
-      versions.put(Version.GFE_61, ExecuteCQ61.getCommand());
-      CommandInitializer.registerCommand(MessageType.EXECUTECQ_MSG_TYPE, versions);
-      CommandInitializer.registerCommand(MessageType.EXECUTECQ_WITH_IR_MSG_TYPE, versions);
-    }
+    Map<Version, Command> versions = new HashMap<>();
+    versions.put(Version.GFE_57, ExecuteCQ.getCommand());
+    versions.put(Version.GFE_61, ExecuteCQ61.getCommand());
+    CommandInitializer.registerCommand(MessageType.EXECUTECQ_MSG_TYPE, versions);
+    CommandInitializer.registerCommand(MessageType.EXECUTECQ_WITH_IR_MSG_TYPE, versions);
 
     CommandInitializer.registerCommand(MessageType.GETCQSTATS_MSG_TYPE,
         Collections.singletonMap(Version.GFE_57, GetCQStats.getCommand()));
@@ -58,7 +57,7 @@ public class CqServiceFactoryImpl implements CqServiceFactory {
   }
 
   @Override
-  public CqService create(GemFireCacheImpl cache) {
+  public CqService create(InternalCache cache) {
     return new CqServiceImpl(cache);
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceImpl.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceImpl.java b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceImpl.java
index f1ca832..570c06c 100644
--- a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceImpl.java
+++ b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceImpl.java
@@ -14,19 +14,63 @@
  */
 package org.apache.geode.cache.query.internal.cq;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.InvalidDeltaException;
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.SystemFailure;
-import org.apache.geode.cache.*;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheEvent;
+import org.apache.geode.cache.CacheLoaderException;
+import org.apache.geode.cache.EntryEvent;
+import org.apache.geode.cache.Operation;
+import org.apache.geode.cache.RegionEvent;
+import org.apache.geode.cache.TimeoutException;
 import org.apache.geode.cache.client.Pool;
-import org.apache.geode.cache.client.internal.*;
-import org.apache.geode.cache.query.*;
-import org.apache.geode.cache.query.internal.*;
+import org.apache.geode.cache.client.internal.GetEventValueOp;
+import org.apache.geode.cache.client.internal.InternalPool;
+import org.apache.geode.cache.client.internal.QueueManager;
+import org.apache.geode.cache.client.internal.ServerCQProxyImpl;
+import org.apache.geode.cache.client.internal.UserAttributes;
+import org.apache.geode.cache.query.CqAttributes;
+import org.apache.geode.cache.query.CqClosedException;
+import org.apache.geode.cache.query.CqException;
+import org.apache.geode.cache.query.CqExistsException;
+import org.apache.geode.cache.query.CqListener;
+import org.apache.geode.cache.query.CqQuery;
+import org.apache.geode.cache.query.CqServiceStatistics;
+import org.apache.geode.cache.query.CqStatusListener;
+import org.apache.geode.cache.query.QueryException;
+import org.apache.geode.cache.query.QueryInvalidException;
+import org.apache.geode.cache.query.RegionNotFoundException;
+import org.apache.geode.cache.query.SelectResults;
+import org.apache.geode.cache.query.internal.CompiledSelect;
+import org.apache.geode.cache.query.internal.CqQueryVsdStats;
+import org.apache.geode.cache.query.internal.CqStateImpl;
+import org.apache.geode.cache.query.internal.DefaultQuery;
+import org.apache.geode.cache.query.internal.ExecutionContext;
 import org.apache.geode.distributed.internal.DistributionAdvisor.Profile;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.i18n.StringId;
 import org.apache.geode.internal.cache.CacheDistributionAdvisor.CacheProfile;
-import org.apache.geode.internal.cache.*;
+import org.apache.geode.internal.cache.EntryEventImpl;
+import org.apache.geode.internal.cache.EventID;
+import org.apache.geode.internal.cache.FilterProfile;
+import org.apache.geode.internal.cache.FilterRoutingInfo;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.tier.MessageType;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientProxy;
@@ -35,57 +79,43 @@ import org.apache.geode.internal.cache.tier.sockets.Part;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
-import org.apache.logging.log4j.Logger;
-
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
 
 /**
- * @since GemFire 5.5
- *
- *        Implements the CqService functionality.
- * 
- */
-/**
+ * Implements the CqService functionality.
  *
+ * @since GemFire 5.5
  */
 public final class CqServiceImpl implements CqService {
   private static final Logger logger = LogService.getLogger();
 
-  private static final Integer MESSAGE_TYPE_LOCAL_CREATE =
-      Integer.valueOf(MessageType.LOCAL_CREATE);
-  private static final Integer MESSAGE_TYPE_LOCAL_UPDATE =
-      Integer.valueOf(MessageType.LOCAL_UPDATE);
-  private static final Integer MESSAGE_TYPE_LOCAL_DESTROY =
-      Integer.valueOf(MessageType.LOCAL_DESTROY);
-  private static final Integer MESSAGE_TYPE_EXCEPTION = Integer.valueOf(MessageType.EXCEPTION);
+  private static final Integer MESSAGE_TYPE_LOCAL_CREATE = MessageType.LOCAL_CREATE;
+  private static final Integer MESSAGE_TYPE_LOCAL_UPDATE = MessageType.LOCAL_UPDATE;
+  private static final Integer MESSAGE_TYPE_LOCAL_DESTROY = MessageType.LOCAL_DESTROY;
+  private static final Integer MESSAGE_TYPE_EXCEPTION = MessageType.EXCEPTION;
 
   /**
    * System property to evaluate the query even though the initial results are not required when cq
    * is executed using the execute() method.
    */
-  public static boolean EXECUTE_QUERY_DURING_INIT = Boolean
-      .valueOf(System
-          .getProperty(DistributionConfig.GEMFIRE_PREFIX + "cq.EXECUTE_QUERY_DURING_INIT", "true"))
-      .booleanValue();
+  public static boolean EXECUTE_QUERY_DURING_INIT = Boolean.valueOf(System
+      .getProperty(DistributionConfig.GEMFIRE_PREFIX + "cq.EXECUTE_QUERY_DURING_INIT", "true"));
 
   private static final String CQ_NAME_PREFIX = "GfCq";
 
-  private final Cache cache;
+  private final InternalCache cache;
 
   /**
    * Manages cq pools to determine if a status of connect or disconnect needs to be sent out
    */
-  private final HashMap<String, Boolean> cqPoolsConnected = new HashMap<String, Boolean>();
-
+  private final HashMap<String, Boolean> cqPoolsConnected = new HashMap<>();
 
   /**
    * Manages CQ objects. uses serverCqName as key and CqQueryImpl as value
    * 
-   * @guarded.By cqQueryMapLock
+   * GuardedBy cqQueryMapLock
    */
-  private volatile HashMap<String, CqQueryImpl> cqQueryMap = new HashMap<String, CqQueryImpl>();
+  private volatile HashMap<String, CqQueryImpl> cqQueryMap = new HashMap<>();
+
   private final Object cqQueryMapLock = new Object();
 
   private volatile boolean isRunning = false;
@@ -93,36 +123,21 @@ public final class CqServiceImpl implements CqService {
   /**
    * Used by client when multiuser-authentication is true.
    */
-  private final HashMap<String, UserAttributes> cqNameToUserAttributesMap =
-      new HashMap<String, UserAttributes>();
-
-  // private boolean isServer = true;
-
-  /*
-   * // Map to manage CQ to satisfied CQ events (keys) for optimizing updates. private final HashMap
-   * cqToCqEventKeysMap = CqService.MAINTAIN_KEYS ? new HashMap() : null;
-   */
+  private final HashMap<String, UserAttributes> cqNameToUserAttributesMap = new HashMap<>();
 
   // Map to manage the similar CQs (having same query - performance optimization).
   // With query as key and Set of CQs as values.
   private final ConcurrentHashMap matchingCqMap;
 
   // CQ Service statistics
-  public final CqServiceStatisticsImpl cqServiceStats;
-  public final CqServiceVsdStats stats;
+  private final CqServiceStatisticsImpl cqServiceStats;
+  private final CqServiceVsdStats stats;
 
   // CQ identifier, also used in auto generated CQ names
   private volatile long cqId = 1;
 
-  /**
-   * Used to synchronize access to CQs in the repository
-   */
-  final Object cqSync = new Object();
-
   /* This is to manage region to CQs map, client side book keeping. */
-  private HashMap<String, ArrayList<String>> baseRegionToCqNameMap =
-      new HashMap<String, ArrayList<String>>();
-
+  private HashMap<String, ArrayList<String>> baseRegionToCqNameMap = new HashMap<>();
 
   /**
    * Access and modification to the contents of this map do not necessarily need to be lock
@@ -135,33 +150,24 @@ public final class CqServiceImpl implements CqService {
 
   /**
    * Constructor.
-   * 
-   * @param c The cache used for the service
+   *
+   * @param cache The cache used for the service
    */
-  public CqServiceImpl(final Cache c) {
-    if (c == null) {
+  public CqServiceImpl(final InternalCache cache) {
+    if (cache == null) {
       throw new IllegalStateException(LocalizedStrings.CqService_CACHE_IS_NULL.toLocalizedString());
     }
-    GemFireCacheImpl gfc = (GemFireCacheImpl) c;
-    gfc.getCancelCriterion().checkCancelInProgress(null);
-
-    this.cache = gfc;
+    cache.getCancelCriterion().checkCancelInProgress(null);
 
+    this.cache = cache;
 
     // Initialize the Map which maintains the matching cqs.
     this.matchingCqMap = new ConcurrentHashMap<String, HashSet<String>>();
 
     // Initialize the VSD statistics
-    StatisticsFactory factory = cache.getDistributedSystem();
+    StatisticsFactory factory = this.cache.getDistributedSystem();
     this.stats = new CqServiceVsdStats(factory);
     this.cqServiceStats = new CqServiceStatisticsImpl(this);
-
-    // final LoggingThreadGroup group =
-    // LoggingThreadGroup.createThreadGroup("CqExecutor Threads", logger);
-
-    // if (this.cache.getCacheServers().isEmpty()) {
-    // isServer = false;
-    // }
   }
 
   /**
@@ -171,13 +177,14 @@ public final class CqServiceImpl implements CqService {
     return this.cache;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#newCq(java.lang.String,
-   * java.lang.String, org.apache.geode.cache.query.CqAttributes,
-   * org.apache.geode.cache.client.internal.ServerCQProxy, boolean)
-   */
+  public InternalCache getInternalCache() {
+    return this.cache;
+  }
+
+  public CqServiceVsdStats stats() {
+    return this.stats;
+  }
+
   @Override
   public synchronized ClientCQ newCq(String cqName, String queryString, CqAttributes cqAttributes,
       InternalPool pool, boolean isDurable)
@@ -242,22 +249,15 @@ public final class CqServiceImpl implements CqService {
     return cQuery;
   }
 
-
   /**
    * Executes the given CqQuery, if the CqQuery for that name is not there it registers the one and
    * executes. This is called on the Server.
    * 
-   * @param cqName
-   * @param queryString
-   * @param cqState
-   * @param clientProxyId
-   * @param ccn
    * @param manageEmptyRegions whether to update the 6.1 emptyRegions map held in the CCN
    * @param regionDataPolicy the data policy of the region associated with the query. This is only
    *        needed if manageEmptyRegions is true.
    * @param emptyRegionsMap map of empty regions.
    * @throws IllegalStateException if this is called at client side.
-   * @throws CqException
    */
   @Override
   public synchronized ServerCQ executeCq(String cqName, String queryString, int cqState,
@@ -271,7 +271,7 @@ public final class CqServiceImpl implements CqService {
     }
 
     String serverCqName = constructServerCqName(cqName, clientProxyId);
-    ServerCQImpl cQuery = null;
+    ServerCQImpl cQuery;
 
     // If this CQ is not yet registered in Server, register CQ.
     if (!isCqExists(serverCqName)) {
@@ -292,7 +292,6 @@ public final class CqServiceImpl implements CqService {
         logger.info(LocalizedMessage.create(
             LocalizedStrings.CqService_EXCEPTION_WHILE_REGISTERING_CQ_ON_SERVER_CQNAME___0,
             cQuery.getName()));
-        cQuery = null;
         throw cqe;
       }
 
@@ -308,6 +307,7 @@ public final class CqServiceImpl implements CqService {
     return cQuery;
   }
 
+  @Override
   public void resumeCQ(int cqState, ServerCQ cQuery) {
     // Initialize the state of CQ.
     if (((CqStateImpl) cQuery.getState()).getState() != cqState) {
@@ -324,25 +324,10 @@ public final class CqServiceImpl implements CqService {
     }
   }
 
-  /*
-   * public void addToCqEventKeysMap(CqQuery cq){ if (cqToCqEventKeysMap != null) { synchronized
-   * (cqToCqEventKeysMap){ String serverCqName = ((CqQueryImpl)cq).getServerCqName(); if
-   * (!cqToCqEventKeysMap.containsKey(serverCqName)){ cqToCqEventKeysMap.put(serverCqName, new
-   * HashSet()); if (_logger.isDebugEnabled()) {
-   * _logger.debug("CQ Event key maintenance for CQ, CqName: " + serverCqName + " is Enabled." +
-   * " key maintenance map size is: " + cqToCqEventKeysMap.size()); } } } // synchronized } }
-   */
-
-  public boolean hasCq() {
-    HashMap<String, CqQueryImpl> cqMap = cqQueryMap;
-    return (cqMap.size() > 0);
-  }
-
-
   /**
    * Adds the given CQ and cqQuery object into the CQ map.
    */
-  public void addToCqMap(CqQueryImpl cq) throws CqExistsException, CqException {
+  void addToCqMap(CqQueryImpl cq) throws CqExistsException, CqException {
     // On server side cqName will be server side cqName.
     String sCqName = cq.getServerCqName();
     if (logger.isDebugEnabled()) {
@@ -355,7 +340,7 @@ public final class CqServiceImpl implements CqService {
               .toLocalizedString(sCqName));
     }
     synchronized (cqQueryMapLock) {
-      HashMap<String, CqQueryImpl> tmpCqQueryMap = new HashMap<String, CqQueryImpl>(cqQueryMap);
+      HashMap<String, CqQueryImpl> tmpCqQueryMap = new HashMap<>(cqQueryMap);
       try {
         tmpCqQueryMap.put(sCqName, cq);
       } catch (Exception ex) {
@@ -377,66 +362,34 @@ public final class CqServiceImpl implements CqService {
   /**
    * Removes given CQ from the cqMap..
    */
-  public void removeCq(String cqName) {
+  void removeCq(String cqName) {
     // On server side cqName will be server side cqName.
     synchronized (cqQueryMapLock) {
-      HashMap<String, CqQueryImpl> tmpCqQueryMap = new HashMap<String, CqQueryImpl>(cqQueryMap);
+      HashMap<String, CqQueryImpl> tmpCqQueryMap = new HashMap<>(cqQueryMap);
       tmpCqQueryMap.remove(cqName);
       this.cqNameToUserAttributesMap.remove(cqName);
       cqQueryMap = tmpCqQueryMap;
     }
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqService#getClientCqFromServer(org.apache.geode.
-   * internal.cache.tier.sockets.ClientProxyMembershipID, java.lang.String)
-   */
   @Override
   public CqQuery getClientCqFromServer(ClientProxyMembershipID clientProxyId, String clientCqName) {
     // On server side cqName will be server side cqName.
     HashMap<String, CqQueryImpl> cqMap = cqQueryMap;
-    return (CqQuery) cqMap.get(this.constructServerCqName(clientCqName, clientProxyId));
+    return cqMap.get(this.constructServerCqName(clientCqName, clientProxyId));
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#getCq(java.lang.String)
-   */
   @Override
   public InternalCqQuery getCq(String cqName) {
     // On server side cqName will be server side cqName.
-    return (InternalCqQuery) cqQueryMap.get(cqName);
+    return cqQueryMap.get(cqName);
   }
 
-  /**
-   * Clears the CQ Query Map.
-   */
-  public void clearCqQueryMap() {
-    // On server side cqName will be server side cqName.
-    synchronized (cqQueryMapLock) {
-      cqQueryMap = new HashMap<String, CqQueryImpl>();
-    }
-  }
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#getAllCqs()
-   */
   @Override
   public Collection<? extends InternalCqQuery> getAllCqs() {
     return cqQueryMap.values();
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#getAllCqs(java.lang.String)
-   */
   @Override
   public Collection<? extends InternalCqQuery> getAllCqs(final String regionName)
       throws CqException {
@@ -445,7 +398,7 @@ public final class CqServiceImpl implements CqService {
           LocalizedStrings.CqService_NULL_ARGUMENT_0.toLocalizedString("regionName"));
     }
 
-    String[] cqNames = null;
+    String[] cqNames;
 
     synchronized (this.baseRegionToCqNameMap) {
       ArrayList<String> cqs = this.baseRegionToCqNameMap.get(regionName);
@@ -456,7 +409,7 @@ public final class CqServiceImpl implements CqService {
       cqs.toArray(cqNames);
     }
 
-    ArrayList<InternalCqQuery> cQueryList = new ArrayList<InternalCqQuery>();
+    ArrayList<InternalCqQuery> cQueryList = new ArrayList<>();
     for (int cqCnt = 0; cqCnt < cqNames.length; cqCnt++) {
       InternalCqQuery cq = getCq(cqNames[cqCnt]);
       if (cq != null) {
@@ -467,34 +420,16 @@ public final class CqServiceImpl implements CqService {
     return cQueryList;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#executeAllClientCqs()
-   */
   @Override
   public synchronized void executeAllClientCqs() throws CqException {
     executeCqs(this.getAllCqs());
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqService#executeAllRegionCqs(java.lang.String)
-   */
   @Override
   public synchronized void executeAllRegionCqs(final String regionName) throws CqException {
     executeCqs(getAllCqs(regionName));
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqService#executeCqs(org.apache.geode.cache.query
-   * .CqQuery[])
-   */
   @Override
   public synchronized void executeCqs(Collection<? extends InternalCqQuery> cqs)
       throws CqException {
@@ -503,53 +438,31 @@ public final class CqServiceImpl implements CqService {
     }
     String cqName = null;
     for (InternalCqQuery internalCq : cqs) {
-      CqQuery cq = (CqQuery) internalCq;
+      CqQuery cq = internalCq;
       if (!cq.isClosed() && cq.isStopped()) {
         try {
           cqName = cq.getName();
           cq.execute();
-        } catch (QueryException qe) {
-          if (logger.isDebugEnabled()) {
-            logger.debug("Failed to execute the CQ, CqName : {} Error : {}", cqName,
-                qe.getMessage());
-          }
-        } catch (CqClosedException cce) {
+        } catch (QueryException | CqClosedException e) {
           if (logger.isDebugEnabled()) {
             logger.debug("Failed to execute the CQ, CqName : {} Error : {}", cqName,
-                cce.getMessage());
+                e.getMessage());
           }
         }
       }
     }
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#stopAllClientCqs()
-   */
   @Override
   public synchronized void stopAllClientCqs() throws CqException {
     stopCqs(this.getAllCqs());
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#stopAllRegionCqs(java.lang.String)
-   */
   @Override
   public synchronized void stopAllRegionCqs(final String regionName) throws CqException {
     stopCqs(this.getAllCqs(regionName));
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqService#stopCqs(org.apache.geode.cache.query.
-   * CqQuery[])
-   */
   @Override
   public synchronized void stopCqs(Collection<? extends InternalCqQuery> cqs) throws CqException {
     final boolean isDebugEnabled = logger.isDebugEnabled();
@@ -567,29 +480,20 @@ public final class CqServiceImpl implements CqService {
 
     String cqName = null;
     for (InternalCqQuery internalCqQuery : cqs) {
-      CqQuery cq = (CqQuery) internalCqQuery;
+      CqQuery cq = internalCqQuery;
       if (!cq.isClosed() && cq.isRunning()) {
         try {
           cqName = cq.getName();
           cq.stop();
-        } catch (QueryException qe) {
-          if (isDebugEnabled) {
-            logger.debug("Failed to stop the CQ, CqName : {} Error : {}", cqName, qe.getMessage());
-          }
-        } catch (CqClosedException cce) {
+        } catch (QueryException | CqClosedException e) {
           if (isDebugEnabled) {
-            logger.debug("Failed to stop the CQ, CqName : {} Error : {}", cqName, cce.getMessage());
+            logger.debug("Failed to stop the CQ, CqName : {} Error : {}", cqName, e.getMessage());
           }
         }
       }
     }
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#closeCqs(java.lang.String)
-   */
   @Override
   public void closeCqs(final String regionName) throws CqException {
     Collection<? extends InternalCqQuery> cqs = this.getAllCqs(regionName);
@@ -603,8 +507,8 @@ public final class CqServiceImpl implements CqService {
             // invoked on the server
             cq.close(false);
           } else {
-            // @todo grid: if regionName has a pool check its keepAlive
-            boolean keepAlive = ((GemFireCacheImpl) this.cache).keepDurableSubscriptionsAlive();
+            // TODO: grid: if regionName has a pool check its keepAlive
+            boolean keepAlive = this.cache.keepDurableSubscriptionsAlive();
             if (cq.isDurable() && keepAlive) {
               logger.warn(LocalizedMessage.create(
                   LocalizedStrings.CqService_NOT_SENDING_CQ_CLOSE_TO_THE_SERVER_AS_IT_IS_A_DURABLE_CQ));
@@ -614,14 +518,9 @@ public final class CqServiceImpl implements CqService {
             }
           }
 
-        } catch (QueryException qe) {
+        } catch (QueryException | CqClosedException e) {
           if (logger.isDebugEnabled()) {
-            logger.debug("Failed to close the CQ, CqName : {} Error : {}", cqName, qe.getMessage());
-          }
-        } catch (CqClosedException cce) {
-          if (logger.isDebugEnabled()) {
-            logger.debug("Failed to close the CQ, CqName : {} Error : {}", cqName,
-                cce.getMessage());
+            logger.debug("Failed to close the CQ, CqName : {} Error : {}", cqName, e.getMessage());
           }
         }
       }
@@ -630,10 +529,6 @@ public final class CqServiceImpl implements CqService {
 
   /**
    * Called directly on server side.
-   * 
-   * @param cqName
-   * @param clientId
-   * @throws CqException
    */
   @Override
   public void stopCq(String cqName, ClientProxyMembershipID clientId) throws CqException {
@@ -650,8 +545,6 @@ public final class CqServiceImpl implements CqService {
     try {
       HashMap<String, CqQueryImpl> cqMap = cqQueryMap;
       if (!cqMap.containsKey(serverCqName)) {
-        // throw new
-        // CqException(LocalizedStrings.CqService_CQ_NOT_FOUND_FAILED_TO_STOP_THE_SPECIFIED_CQ_0.toLocalizedString(serverCqName));
         /*
          * gregp 052808: We should silently fail here instead of throwing error. This is to deal
          * with races in recovery
@@ -689,15 +582,8 @@ public final class CqServiceImpl implements CqService {
     }
     // Send stop message to peers.
     cQuery.getCqBaseRegion().getFilterProfile().stopCq(cQuery);
-
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#closeCq(java.lang.String,
-   * org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID)
-   */
   @Override
   public void closeCq(String cqName, ClientProxyMembershipID clientProxyId) throws CqException {
     String serverCqName = cqName;
@@ -713,9 +599,6 @@ public final class CqServiceImpl implements CqService {
     try {
       HashMap<String, CqQueryImpl> cqMap = cqQueryMap;
       if (!cqMap.containsKey(serverCqName)) {
-        // throw new
-        // CqException(LocalizedStrings.CqService_CQ_NOT_FOUND_FAILED_TO_CLOSE_THE_SPECIFIED_CQ_0
-        // .toLocalizedString(serverCqName));
         /*
          * gregp 052808: We should silently fail here instead of throwing error. This is to deal
          * with races in recovery
@@ -791,12 +674,6 @@ public final class CqServiceImpl implements CqService {
     }
   }
 
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#closeAllCqs(boolean)
-   */
   @Override
   public void closeAllCqs(boolean clientInitiated) {
     closeAllCqs(clientInitiated, getAllCqs());
@@ -807,21 +684,13 @@ public final class CqServiceImpl implements CqService {
    * CqQuerys created by other VMs are unaffected.
    */
   private void closeAllCqs(boolean clientInitiated, Collection<? extends InternalCqQuery> cqs) {
-    closeAllCqs(clientInitiated, cqs,
-        ((GemFireCacheImpl) this.cache).keepDurableSubscriptionsAlive());
+    closeAllCqs(clientInitiated, cqs, this.cache.keepDurableSubscriptionsAlive());
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#closeAllCqs(boolean,
-   * org.apache.geode.cache.query.CqQuery[], boolean)
-   */
   @Override
   public void closeAllCqs(boolean clientInitiated, Collection<? extends InternalCqQuery> cqs,
       boolean keepAlive) {
 
-    // CqQuery[] cqs = getAllCqs();
     if (cqs != null) {
       String cqName = null;
       if (logger.isDebugEnabled()) {
@@ -830,7 +699,6 @@ public final class CqServiceImpl implements CqService {
       for (InternalCqQuery cQuery : cqs) {
         try {
           cqName = cQuery.getName();
-          // boolean keepAlive = ((GemFireCache)this.cache).keepDurableSubscriptionsAlive();
 
           if (isServer()) {
             cQuery.close(false);
@@ -847,47 +715,26 @@ public final class CqServiceImpl implements CqService {
               }
             }
           }
-        } catch (QueryException cqe) {
+        } catch (QueryException | CqClosedException e) {
           if (!isRunning()) {
             // Not cache shutdown
             logger
                 .warn(LocalizedMessage.create(LocalizedStrings.CqService_FAILED_TO_CLOSE_CQ__0___1,
-                    new Object[] {cqName, cqe.getMessage()}));
+                    new Object[] {cqName, e.getMessage()}));
           }
           if (logger.isDebugEnabled()) {
-            logger.debug(cqe.getMessage(), cqe);
-          }
-        } catch (CqClosedException cqe) {
-          if (!isRunning()) {
-            // Not cache shutdown
-            logger
-                .warn(LocalizedMessage.create(LocalizedStrings.CqService_FAILED_TO_CLOSE_CQ__0___1,
-                    new Object[] {cqName, cqe.getMessage()}));
-          }
-          if (logger.isDebugEnabled()) {
-            logger.debug(cqe.getMessage(), cqe);
+            logger.debug(e.getMessage(), e);
           }
         }
       }
     }
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#getCqStatistics()
-   */
   @Override
   public CqServiceStatistics getCqStatistics() {
     return cqServiceStats;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#closeClientCqs(org.apache.geode.
-   * internal.cache.tier.sockets.ClientProxyMembershipID)
-   */
   @Override
   public void closeClientCqs(ClientProxyMembershipID clientProxyId) throws CqException {
     final boolean isDebugEnabled = logger.isDebugEnabled();
@@ -899,30 +746,19 @@ public final class CqServiceImpl implements CqService {
       CqQueryImpl cQuery = (CqQueryImpl) cq;
       try {
         cQuery.close(false);
-      } catch (QueryException qe) {
+      } catch (QueryException | CqClosedException e) {
         if (isDebugEnabled) {
           logger.debug("Failed to close the CQ, CqName : {} Error : {}", cQuery.getName(),
-              qe.getMessage());
-        }
-      } catch (CqClosedException cce) {
-        if (isDebugEnabled) {
-          logger.debug("Failed to close the CQ, CqName : {} Error : {}", cQuery.getName(),
-              cce.getMessage());
+              e.getMessage());
         }
       }
     }
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqService#getAllClientCqs(org.apache.geode.
-   * internal.cache.tier.sockets.ClientProxyMembershipID)
-   */
   @Override
   public List<ServerCQ> getAllClientCqs(ClientProxyMembershipID clientProxyId) {
     Collection<? extends InternalCqQuery> cqs = getAllCqs();
-    ArrayList<ServerCQ> clientCqs = new ArrayList<ServerCQ>();
+    ArrayList<ServerCQ> clientCqs = new ArrayList<>();
 
     for (InternalCqQuery cq : cqs) {
       ServerCQImpl cQuery = (ServerCQImpl) cq;
@@ -934,23 +770,16 @@ public final class CqServiceImpl implements CqService {
     return clientCqs;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqService#getAllDurableClientCqs(org.apache.geode
-   * .internal.cache.tier.sockets.ClientProxyMembershipID)
-   */
   @Override
   public List<String> getAllDurableClientCqs(ClientProxyMembershipID clientProxyId)
       throws CqException {
     if (clientProxyId == null) {
       throw new CqException(
           LocalizedStrings.CqService_UNABLE_TO_RETRIEVE_DURABLE_CQS_FOR_CLIENT_PROXY_ID
-              .toLocalizedString(clientProxyId));
+              .toLocalizedString(null));
     }
     List<ServerCQ> cqs = getAllClientCqs(clientProxyId);
-    ArrayList<String> durableClientCqs = new ArrayList<String>();
+    ArrayList<String> durableClientCqs = new ArrayList<>();
 
     for (ServerCQ cq : cqs) {
       ServerCQImpl cQuery = (ServerCQImpl) cq;
@@ -966,9 +795,6 @@ public final class CqServiceImpl implements CqService {
 
   /**
    * Server side method. Closes non-durable CQs for the given client proxy id.
-   * 
-   * @param clientProxyId
-   * @throws CqException
    */
   @Override
   public void closeNonDurableClientCqs(ClientProxyMembershipID clientProxyId) throws CqException {
@@ -983,15 +809,10 @@ public final class CqServiceImpl implements CqService {
         if (!cQuery.isDurable()) {
           cQuery.close(false);
         }
-      } catch (QueryException qe) {
-        if (isDebugEnabled) {
-          logger.debug("Failed to close the CQ, CqName : {} Error : {}", cQuery.getName(),
-              qe.getMessage());
-        }
-      } catch (CqClosedException cce) {
+      } catch (QueryException | CqClosedException e) {
         if (isDebugEnabled) {
           logger.debug("Failed to close the CQ, CqName : {} Error : {}", cQuery.getName(),
-              cce.getMessage());
+              e.getMessage());
         }
       }
     }
@@ -1028,6 +849,7 @@ public final class CqServiceImpl implements CqService {
     return this.isRunning;
   }
 
+  @Override
   public void start() {
     this.isRunning = true;
   }
@@ -1035,9 +857,10 @@ public final class CqServiceImpl implements CqService {
   /**
    * @return Returns the serverCqName.
    */
+  @Override
   public String constructServerCqName(String cqName, ClientProxyMembershipID clientProxyId) {
-    ConcurrentHashMap<ClientProxyMembershipID, String> cache = serverCqNameCache
-        .computeIfAbsent(cqName, key -> new ConcurrentHashMap<ClientProxyMembershipID, String>());
+    ConcurrentHashMap<ClientProxyMembershipID, String> cache =
+        serverCqNameCache.computeIfAbsent(cqName, key -> new ConcurrentHashMap<>());
 
     String cName = cache.get(clientProxyId);
     if (null == cName) {
@@ -1065,7 +888,7 @@ public final class CqServiceImpl implements CqService {
     }
   }
 
-  /*
+  /**
    * Checks if CQ with the given name already exists.
    * 
    * @param cqName name of the CQ.
@@ -1073,17 +896,15 @@ public final class CqServiceImpl implements CqService {
    * @return true if exists else false.
    */
   private synchronized boolean isCqExists(String cqName) {
-    boolean status = false;
     HashMap<String, CqQueryImpl> cqMap = cqQueryMap;
-    status = cqMap.containsKey(cqName);
-    return status;
+    return cqMap.containsKey(cqName);
   }
 
-  /*
+  /**
    * Generates a name for CQ. Checks if CQ with that name already exists if so generates a new
    * cqName.
    */
-  public synchronized String generateCqName() {
+  private synchronized String generateCqName() {
     while (true) {
       String cqName = CQ_NAME_PREFIX + (cqId++);
       if (!isCqExists(cqName)) {
@@ -1092,18 +913,9 @@ public final class CqServiceImpl implements CqService {
     }
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqService#dispatchCqListeners(java.util.HashMap,
-   * int, java.lang.Object, java.lang.Object, byte[],
-   * org.apache.geode.cache.client.internal.QueueManager, org.apache.geode.internal.cache.EventID)
-   */
   @Override
   public void dispatchCqListeners(HashMap<String, Integer> cqs, int messageType, Object key,
       Object value, byte[] delta, QueueManager qManager, EventID eventId) {
-    ClientCQImpl cQuery = null;
     Object[] fullValue = new Object[1];
     Iterator<Map.Entry<String, Integer>> iter = cqs.entrySet().iterator();
     String cqName = null;
@@ -1112,7 +924,7 @@ public final class CqServiceImpl implements CqService {
       try {
         Map.Entry<String, Integer> entry = iter.next();
         cqName = entry.getKey();
-        cQuery = (ClientCQImpl) this.getCq(cqName);
+        ClientCQImpl cQuery = (ClientCQImpl) this.getCq(cqName);
 
         if (cQuery == null || (!cQuery.isRunning() && cQuery.getQueuedEvents() == null)) {
           if (isDebugEnabled) {
@@ -1122,7 +934,7 @@ public final class CqServiceImpl implements CqService {
           continue;
         }
 
-        Integer cqOp = (Integer) entry.getValue();
+        Integer cqOp = entry.getValue();
 
         // If Region destroy event, close the cq.
         if (cqOp.intValue() == MessageType.DESTROY_REGION) {
@@ -1136,8 +948,7 @@ public final class CqServiceImpl implements CqService {
         }
 
         // Construct CqEvent.
-        CqEventImpl cqEvent = null;
-        cqEvent = new CqEventImpl(cQuery, getOperation(messageType), getOperation(cqOp.intValue()),
+        CqEventImpl cqEvent = new CqEventImpl(cQuery, getOperation(messageType), getOperation(cqOp),
             key, value, delta, qManager, eventId);
 
         // Update statistics
@@ -1181,11 +992,11 @@ public final class CqServiceImpl implements CqService {
     } // iteration.
   }
 
-  public void invokeListeners(String cqName, ClientCQImpl cQuery, CqEventImpl cqEvent) {
+  void invokeListeners(String cqName, ClientCQImpl cQuery, CqEventImpl cqEvent) {
     invokeListeners(cqName, cQuery, cqEvent, null);
   }
 
-  public void invokeListeners(String cqName, ClientCQImpl cQuery, CqEventImpl cqEvent,
+  private void invokeListeners(String cqName, ClientCQImpl cQuery, CqEventImpl cqEvent,
       Object[] fullValue) {
     if (!cQuery.isRunning() || cQuery.getCqAttributes() == null) {
       return;
@@ -1217,8 +1028,8 @@ public final class CqServiceImpl implements CqService {
             }
             Part result = (Part) GetEventValueOp
                 .executeOnPrimary(cqEvent.getQueueManager().getPool(), cqEvent.getEventID(), null);
-            Object newVal = null;
-            if (result == null || (newVal = result.getObject()) == null) {
+            Object newVal = result.getObject();
+            if (result == null || newVal == null) {
               if (!cache.getCancelCriterion().isCancelInProgress()) {
                 Exception ex =
                     new Exception("Failed to retrieve full value from server for eventID "
@@ -1231,7 +1042,7 @@ public final class CqServiceImpl implements CqService {
                 }
               }
             } else {
-              ((GemFireCacheImpl) this.cache).getCachePerfStats().incDeltaFullValuesRequested();
+              this.cache.getCachePerfStats().incDeltaFullValuesRequested();
               cqEvent = new CqEventImpl(cQuery, cqEvent.getBaseOperation(),
                   cqEvent.getQueryOperation(), cqEvent.getKey(), newVal, cqEvent.getDeltaValue(),
                   cqEvent.getQueueManager(), cqEvent.getEventID());
@@ -1278,7 +1089,7 @@ public final class CqServiceImpl implements CqService {
     }
   }
 
-  public void invokeCqConnectedListeners(String cqName, ClientCQImpl cQuery, boolean connected) {
+  private void invokeCqConnectedListeners(String cqName, ClientCQImpl cQuery, boolean connected) {
     if (!cQuery.isRunning() || cQuery.getCqAttributes() == null) {
       return;
     }
@@ -1335,12 +1146,8 @@ public final class CqServiceImpl implements CqService {
     }
   }
 
-
   /**
    * Returns the Operation for the given EnumListenerEvent type.
-   * 
-   * @param eventType
-   * @return Operation
    */
   private Operation getOperation(int eventType) {
     Operation op = null;
@@ -1372,15 +1179,6 @@ public final class CqServiceImpl implements CqService {
     return op;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqService#processEvents(org.apache.geode.cache.
-   * CacheEvent, org.apache.geode.distributed.internal.DistributionAdvisor.Profile,
-   * org.apache.geode.distributed.internal.DistributionAdvisor.Profile[],
-   * org.apache.geode.internal.cache.FilterRoutingInfo)
-   */
   @Override
   public void processEvents(CacheEvent event, Profile localProfile, Profile[] profiles,
       FilterRoutingInfo frInfo) throws CqException {
@@ -1421,7 +1219,7 @@ public final class CqServiceImpl implements CqService {
         continue;
       }
       Map cqs = pf.getCqMap();
-      HashMap<Long, Integer> cqInfo = new HashMap<Long, Integer>();
+      HashMap<Long, Integer> cqInfo = new HashMap<>();
       Iterator cqIter = cqs.entrySet().iterator();
       while (cqIter.hasNext()) {
         Map.Entry cqEntry = (Map.Entry) cqIter.next();
@@ -1454,10 +1252,10 @@ public final class CqServiceImpl implements CqService {
   private void processEntryEvent(CacheEvent event, Profile localProfile, Profile[] profiles,
       FilterRoutingInfo frInfo) throws CqException {
     final boolean isDebugEnabled = logger.isDebugEnabled();
-    HashSet<Object> cqUnfilteredEventsSet_newValue = new HashSet<Object>();
-    HashSet<Object> cqUnfilteredEventsSet_oldValue = new HashSet<Object>();
-    boolean b_cqResults_newValue = false;
-    boolean b_cqResults_oldValue = false;
+    HashSet<Object> cqUnfilteredEventsSet_newValue = new HashSet<>();
+    HashSet<Object> cqUnfilteredEventsSet_oldValue = new HashSet<>();
+    boolean b_cqResults_newValue;
+    boolean b_cqResults_oldValue;
     boolean queryOldValue;
     EntryEvent entryEvent = (EntryEvent) event;
     Object eventKey = entryEvent.getKey();
@@ -1472,8 +1270,8 @@ public final class CqServiceImpl implements CqService {
         || event.getOperation().isDestroy() || event.getOperation().isInvalidate()
         || (event.getOperation().isCreate() && isDupEvent));
 
-    HashMap<String, Integer> matchedCqs = new HashMap<String, Integer>();
-    long executionStartTime = 0;
+    HashMap<String, Integer> matchedCqs = new HashMap<>();
+    long executionStartTime;
     for (int i = -1; i < profiles.length; i++) {
       CacheProfile cf;
       if (i < 0) {
@@ -1498,7 +1296,6 @@ public final class CqServiceImpl implements CqService {
         continue;
       }
 
-
       // Get new value. If its not retrieved.
       if (cqUnfilteredEventsSet_newValue.isEmpty()
           && (event.getOperation().isCreate() || event.getOperation().isUpdate())) {
@@ -1509,7 +1306,7 @@ public final class CqServiceImpl implements CqService {
         }
       }
 
-      HashMap<Long, Integer> cqInfo = new HashMap<Long, Integer>();
+      HashMap<Long, Integer> cqInfo = new HashMap<>();
       Iterator cqIter = cqs.entrySet().iterator();
 
       while (cqIter.hasNext()) {
@@ -1546,7 +1343,6 @@ public final class CqServiceImpl implements CqService {
           }
         } else {
           boolean error = false;
-          // synchronized (cQuery)
           {
             try {
               synchronized (cQuery) {
@@ -1644,7 +1440,7 @@ public final class CqServiceImpl implements CqService {
                 cQuery.markAsDestroyedInCqResultKeys(eventKey);
               }
             }
-          } // end synchronized(cQuery)
+          }
 
           // Get the matching CQs if any.
           // synchronized (this.matchingCqMap){
@@ -1663,7 +1459,6 @@ public final class CqServiceImpl implements CqService {
               }
             }
           }
-          // }
         }
 
         if (cqEvent != null && cQuery.isRunning()) {
@@ -1694,153 +1489,35 @@ public final class CqServiceImpl implements CqService {
     } // iteration over Profiles.
   }
 
-
-  /*
-   * public void processEvents (EnumListenerEvent operation, CacheEvent event, ClientUpdateMessage
-   * clientMessage, CM<ClientProxyMembershipID, CM<CqQuery, Boolean>> clientIds) throws CqException
-   * {
-   * 
-   * //Is this a region event or an entry event if (event instanceof RegionEvent){
-   * processRegionEvent(operation, event, clientMessage, clientIds); } else { processEntryEvent
-   * (operation, event, clientMessage, clientIds); }
-   * 
-   * }
-   * 
-   * private void processRegionEvent(EnumListenerEvent operation, CacheEvent event,
-   * ClientUpdateMessage clientMessage, CM<ClientProxyMembershipID, CM<CqQuery, Boolean>> clientIds)
-   * throws CqException {
-   * 
-   * if (logger.isDebugEnabled()) { logger.debug("Processing region event for region " +
-   * ((LocalRegion)(event.getRegion())).getName()); } HashMap filteredCqs = new HashMap(); Integer
-   * cqRegionEvent = generateCqRegionEvent(operation); Iterator it =
-   * clientIds.entrySet().iterator(); while (it.hasNext()) { Map.Entry me = (Map.Entry)it.next();
-   * ClientProxyMembershipID clientId = (ClientProxyMembershipID)me.getKey(); CM cqsToBooleans =
-   * (CM)me.getValue(); if (cqsToBooleans == null) { continue; } Set<CqQuery> cqs =
-   * cqsToBooleans.keySet(); if (cqs.isEmpty()) { continue; } filteredCqs.clear(); Iterator cqIt =
-   * cqs.iterator(); while (cqIt.hasNext()) { CqQueryImpl cQuery = (CqQueryImpl)cqIt.next(); if
-   * (operation == EnumListenerEvent.AFTER_REGION_DESTROY) { try { if (logger.isDebugEnabled()){
-   * logger.debug("Closing CQ on region destroy event. CqName :" + cQuery.getName()); }
-   * cQuery.close(false); } catch (Exception ex) {
-   * logger.debug("Failed to Close CQ on region destroy. CqName :" + cQuery.getName(), ex); }
-   * 
-   * } filteredCqs.put(cQuery.cqName, cqRegionEvent);
-   * cQuery.getVsdStats().updateStats(cqRegionEvent);
-   * 
-   * } if (!filteredCqs.isEmpty()){ ((ClientUpdateMessageImpl)clientMessage).addClientCqs( clientId,
-   * filteredCqs); }
-   * 
-   * }
-   * 
-   * }
-   * 
-   * private void processEntryEvent(EnumListenerEvent operation, CacheEvent event,
-   * ClientUpdateMessage clientMessage, CM<ClientProxyMembershipID, CM<CqQuery, Boolean>> clientIds)
-   * throws CqException { HashSet cqUnfilteredEventsSet_newValue = new HashSet(); HashSet
-   * cqUnfilteredEventsSet_oldValue = new HashSet(); boolean b_cqResults_newValue = false; boolean
-   * b_cqResults_oldValue = false; EntryEvent entryEvent = (EntryEvent)event; Object eventKey =
-   * entryEvent.getKey(); if (operation == EnumListenerEvent.AFTER_CREATE || operation ==
-   * EnumListenerEvent.AFTER_UPDATE) { if (entryEvent.getNewValue() != null) { //We have a new value
-   * to run the query on cqUnfilteredEventsSet_newValue.clear();
-   * cqUnfilteredEventsSet_newValue.add(entryEvent.getNewValue()); } }
-   * 
-   * HashMap matchedCqs = new HashMap(); long executionStartTime = 0; Iterator it =
-   * clientIds.entrySet().iterator(); while (it.hasNext()) { Map.Entry me = (Map.Entry)it.next();
-   * ClientProxyMembershipID clientId = (ClientProxyMembershipID)me.getKey(); if
-   * (logger.isDebugEnabled()) { logger.debug("Processing event for CQ filter, ClientId : " +
-   * clientId); } CM cqsToBooleans = (CM)me.getValue(); if (cqsToBooleans == null) { continue; }
-   * Set<CqQuery> cqs = cqsToBooleans.keySet(); if (cqs.isEmpty()) { continue; } HashMap filteredCqs
-   * = new HashMap(); Iterator cqIt = cqs.iterator(); while (cqIt.hasNext()) { CqQueryImpl cQuery =
-   * (CqQueryImpl)cqIt.next(); b_cqResults_newValue = false; b_cqResults_oldValue = false; if
-   * (cQuery == null || !(cQuery.isRunning())){ continue; } String cqName =
-   * cQuery.getServerCqName(); Integer cqEvent = null; if (matchedCqs.containsKey(cqName)) { if
-   * (logger.isDebugEnabled()){ logger.
-   * debug("Similar cq/query is already processed, getting the cq event-type from the matched cq.");
-   * } cqEvent = (Integer)matchedCqs.get(cqName); } else { boolean error = false; boolean
-   * hasSeenEvent = false; HashSet cqEventKeys = null; synchronized (cQuery) { try { // Apply query
-   * on new value. if (!cqUnfilteredEventsSet_newValue.isEmpty()) { executionStartTime =
-   * this.stats.startCqQueryExecution(); b_cqResults_newValue = evaluateQuery(cQuery, new Object[]
-   * {cqUnfilteredEventsSet_newValue}); this.stats.endCqQueryExecution(executionStartTime); } //
-   * Check if old value is cached, if not apply query on old value. if (cqToCqEventKeysMap != null)
-   * { synchronized (cqToCqEventKeysMap) { if ((cqEventKeys =
-   * (HashSet)cqToCqEventKeysMap.get(cqName)) != null) { hasSeenEvent =
-   * cqEventKeys.contains(eventKey); } } } if (!hasSeenEvent) { // get the oldValue. // In case of
-   * Update, destroy and invalidate. if (operation == EnumListenerEvent.AFTER_UPDATE || operation ==
-   * EnumListenerEvent.AFTER_DESTROY || operation == EnumListenerEvent.AFTER_INVALIDATE) { if
-   * (entryEvent.getOldValue() != null) { cqUnfilteredEventsSet_oldValue.clear();
-   * cqUnfilteredEventsSet_oldValue.add(entryEvent.getOldValue()); // Apply query on old value.
-   * executionStartTime = this.stats.startCqQueryExecution(); b_cqResults_oldValue =
-   * evaluateQuery(cQuery, new Object[] {cqUnfilteredEventsSet_oldValue});
-   * this.stats.endCqQueryExecution(executionStartTime); } } } } catch (Exception ex) { //Any
-   * exception in running the query // should be caught here and buried //because this code is
-   * running inline with the //message processing code and we don't want to //kill that thread error
-   * = true; logger.info( LocalizedStrings.
-   * CqService_ERROR_WHILE_PROCESSING_CQ_ON_THE_EVENT_KEY_0_CQNAME_1_CLIENTID_2_ERROR_3, new
-   * Object[] { ((EntryEvent)event).getKey(), cQuery.getName(), clientId,
-   * ex.getLocalizedMessage()}); }
-   * 
-   * if (error) { cqEvent = Integer.valueOf(MessageType.EXCEPTION); } else { if
-   * (b_cqResults_newValue) { if (hasSeenEvent || b_cqResults_oldValue) { cqEvent =
-   * Integer.valueOf(MessageType.LOCAL_UPDATE); } else { cqEvent =
-   * Integer.valueOf(MessageType.LOCAL_CREATE); } // If its create and caching is enabled, cache the
-   * key for this CQ. if (!hasSeenEvent && cqEventKeys != null) { cqEventKeys.add(eventKey); } }
-   * else if (hasSeenEvent || (b_cqResults_oldValue)) { // Base invalidate operation is treated as
-   * destroy. // When the invalidate comes through, the entry will no longer satisfy // the query
-   * and will need to be deleted. cqEvent = Integer.valueOf(MessageType.LOCAL_DESTROY); // If
-   * caching is enabled, remove this event's key from the cache. if (hasSeenEvent && cqEventKeys !=
-   * null) { cqEventKeys.remove(eventKey); } } }
-   * 
-   * } //end synchronized(cQuery)
-   * 
-   * // Get the matching CQs if any. synchronized (this.matchingCqMap){ String query =
-   * cQuery.getQueryString(); ArrayList matchingCqs = (ArrayList)matchingCqMap.get(query); if
-   * (matchingCqs != null) { Iterator iter = matchingCqs.iterator(); while (iter.hasNext()) { String
-   * matchingCqName = (String)iter.next(); if (!matchingCqName.equals(cqName)){
-   * matchedCqs.put(matchingCqName, cqEvent); } } } }
-   * 
-   * }
-   * 
-   * if (cqEvent != null){ if (logger.isDebugEnabled()) {
-   * logger.debug("Event is added for the CQ, CqName (clientside): " + cQuery.cqName +
-   * " With CQ Op : " + cqEvent + " for Client : " + clientId); } filteredCqs.put(cQuery.cqName,
-   * cqEvent); cQuery.getVsdStats().updateStats(cqEvent); }
-   * 
-   * } // iteration over cqsToBooleans.keySet() if (!filteredCqs.isEmpty()){
-   * logger.debug("Adding event map for client : "+clientId +
-   * " with event map size : "+filteredCqs.size());
-   * ((ClientUpdateMessageImpl)clientMessage).addClientCqs(clientId, filteredCqs); } } // iteration
-   * over clientIds.entrySet() }
-   */
-
   private Integer generateCqRegionEvent(CacheEvent event) {
     Integer cqEvent = null;
     if (event.getOperation().isRegionDestroy()) {
-      cqEvent = Integer.valueOf(MessageType.DESTROY_REGION);
+      cqEvent = MessageType.DESTROY_REGION;
     } else if (event.getOperation().isRegionInvalidate()) {
-      cqEvent = Integer.valueOf(MessageType.INVALIDATE_REGION);
+      cqEvent = MessageType.INVALIDATE_REGION;
     } else if (event.getOperation().isClear()) {
-      cqEvent = Integer.valueOf(MessageType.CLEAR_REGION);
+      cqEvent = MessageType.CLEAR_REGION;
     }
     return cqEvent;
   }
 
-
   /**
    * Manages the CQs created for the base region. This is managed here, instead of on the base
    * region; since the cq could be created on the base region, before base region is created (using
    * newCq()).
    */
-  public void addToBaseRegionToCqNameMap(String regionName, String cqName) {
+  private void addToBaseRegionToCqNameMap(String regionName, String cqName) {
     synchronized (this.baseRegionToCqNameMap) {
       ArrayList<String> cqs = this.baseRegionToCqNameMap.get(regionName);
       if (cqs == null) {
-        cqs = new ArrayList<String>();
+        cqs = new ArrayList<>();
       }
       cqs.add(cqName);
       this.baseRegionToCqNameMap.put(regionName, cqs);
     }
   }
 
-  public void removeFromBaseRegionToCqNameMap(String regionName, String cqName) {
+  void removeFromBaseRegionToCqNameMap(String regionName, String cqName) {
     synchronized (this.baseRegionToCqNameMap) {
       ArrayList<String> cqs = this.baseRegionToCqNameMap.get(regionName);
       if (cqs != null) {
@@ -1864,37 +1541,12 @@ public final class CqServiceImpl implements CqService {
   }
 
   /**
-   * Removes this CQ from CQ event Cache map. This disables the caching events for this CQ.
-   * 
-   * @param cqName
-   */
-  /*
-   * synchronized public void removeCQFromCaching(String cqName){ if (cqToCqEventKeysMap != null) {
-   * // Take a lock on CqQuery object. In processEvents the maps are // handled under CqQuery
-   * object. if (cqToCqEventKeysMap != null){ synchronized (cqToCqEventKeysMap) {
-   * cqToCqEventKeysMap.remove(cqName); } } } }
-   */
-
-  /**
-   * Returns the CQ event cache map.
-   * 
-   * @return HashMap cqToCqEventKeysMap
-   * 
-   *         Caller must synchronize on the returned value in order to inspect.
-   */
-  /*
-   * public HashMap getCqToCqEventKeysMap(){ return cqToCqEventKeysMap; }
-   */
-
-  /**
    * Adds the query from the given CQ to the matched CQ map.
-   * 
-   * @param cq
    */
-  public void addToMatchingCqMap(CqQueryImpl cq) {
+  void addToMatchingCqMap(CqQueryImpl cq) {
     synchronized (this.matchingCqMap) {
       String cqQuery = cq.getQueryString();
-      Set<String> matchingCQs = null;
+      Set<String> matchingCQs;
       if (!matchingCqMap.containsKey(cqQuery)) {
         matchingCQs = Collections.newSetFromMap(new ConcurrentHashMap());
         matchingCqMap.put(cqQuery, matchingCQs);
@@ -1912,10 +1564,8 @@ public final class CqServiceImpl implements CqService {
 
   /**
    * Removes the query from the given CQ from the matched CQ map.
-   * 
-   * @param cq
    */
-  public void removeFromMatchingCqMap(CqQueryImpl cq) {
+  private void removeFromMatchingCqMap(CqQueryImpl cq) {
     synchronized (this.matchingCqMap) {
       String cqQuery = cq.getQueryString();
       if (matchingCqMap.containsKey(cqQuery)) {
@@ -1947,10 +1597,6 @@ public final class CqServiceImpl implements CqService {
    * Applies the query on the event. This method takes care of the performance related changed done
    * to improve the CQ-query performance. When CQ-query is executed first time, it saves the query
    * related information in the execution context and uses that info in later executions.
-   * 
-   * @param cQuery
-   * @param event
-   * @return boolean
    */
   private boolean evaluateQuery(CqQueryImpl cQuery, Object[] event) throws Exception {
     ExecutionContext execContext = cQuery.getQueryExecutionContext();
@@ -1983,19 +1629,6 @@ public final class CqServiceImpl implements CqService {
     return this.cqNameToUserAttributesMap.get(cqName);
   }
 
-  // public static void memberLeft(String poolName) {
-  // if (cqServiceSingleton != null && !cqServiceSingleton.isServer()) {
-  // cqServiceSingleton.sendMemberDisconnectedMessageToCqs(poolName);
-  // }
-  // }
-  //
-  // public static void memberCrashed(String poolName) {
-  // if (cqServiceSingleton != null && !cqServiceSingleton.isServer()) {
-  // cqServiceSingleton.sendMemberDisconnectedMessageToCqs(poolName);
-  // }
-  // }
-  //
-
   @Override
   public void cqsDisconnected(Pool pool) {
     invokeCqsConnected(pool, false);
@@ -2014,7 +1647,7 @@ public final class CqServiceImpl implements CqService {
     // Check to see if we are already connected/disconnected.
     // If state has not changed, do not invoke another connected/disconnected
     synchronized (cqPoolsConnected) {
-      // don't repeatily send same connect/disconnect message to cq's on repeated fails of
+      // don't repeatedly send same connect/disconnect message to cq's on repeated fails of
       // RedundancySatisfier
       if (cqPoolsConnected.containsKey(poolName) && connected == cqPoolsConnected.get(poolName)) {
         return;
@@ -2059,13 +1692,6 @@ public final class CqServiceImpl implements CqService {
           SystemFailure.checkFailure();
           logger.warn(LocalizedMessage
               .create(LocalizedStrings.CqService_ERROR_SENDING_CQ_CONNECTION_STATUS, cqName), t);
-
-          if (t instanceof VirtualMachineError) {
-            logger.warn(LocalizedMessage.create(
-                LocalizedStrings.CqService_VIRTUALMACHINEERROR_PROCESSING_CQLISTENER_FOR_CQ_0,
-                cqName), t);
-            return;
-          }
         }
       }
     }
@@ -2075,7 +1701,4 @@ public final class CqServiceImpl implements CqService {
   public List<String> getAllDurableCqsFromServer(InternalPool pool) {
     return new ServerCQProxyImpl(pool).getAllDurableCqsFromServer();
   }
-
-
 }
-

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceStatisticsImpl.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceStatisticsImpl.java b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceStatisticsImpl.java
index ba71143..a675162 100644
--- a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceStatisticsImpl.java
+++ b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceStatisticsImpl.java
@@ -14,11 +14,9 @@
  */
 package org.apache.geode.cache.query.internal.cq;
 
-import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.query.CqServiceStatistics;
 import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.internal.DefaultQueryService;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 
 /**
  * Provides statistical information about CqService.
@@ -26,24 +24,22 @@ import org.apache.geode.internal.cache.GemFireCacheImpl;
  * @since GemFire 5.5
  */
 public class CqServiceStatisticsImpl implements CqServiceStatistics {
+
   private CqServiceImpl cqService;
-  // private long activeCqs;
-  // private long stoppedCqs;
-  // private long closedCqs;
-  // private long createdCqs;
 
   /**
    * Constructor for CqStatisticsImpl
    * 
    * @param cqs - CqService
    */
-  public CqServiceStatisticsImpl(CqServiceImpl cqs) {
+  CqServiceStatisticsImpl(CqServiceImpl cqs) {
     cqService = cqs;
   }
 
   /**
    * Returns the number of CQs currently executing
    */
+  @Override
   public long numCqsActive() {
     return this.cqService.getCqServiceVsdStats().getNumCqsActive();
   }
@@ -53,6 +49,7 @@ public class CqServiceStatisticsImpl implements CqServiceStatistics {
    * 
    * @return long number of cqs created.
    */
+  @Override
   public long numCqsCreated() {
     return this.cqService.getCqServiceVsdStats().getNumCqsCreated();
   }
@@ -60,6 +57,7 @@ public class CqServiceStatisticsImpl implements CqServiceStatistics {
   /**
    * Returns number of Cqs that are closed.
    */
+  @Override
   public long numCqsClosed() {
     return this.cqService.getCqServiceVsdStats().getNumCqsClosed();
   }
@@ -67,6 +65,7 @@ public class CqServiceStatisticsImpl implements CqServiceStatistics {
   /**
    * Returns number of Cqs that are stopped.
    */
+  @Override
   public long numCqsStopped() {
     return this.cqService.getCqServiceVsdStats().getNumCqsStopped();
   }
@@ -74,20 +73,18 @@ public class CqServiceStatisticsImpl implements CqServiceStatistics {
   /**
    * Returns number of CQs created from the client.
    */
+  @Override
   public long numCqsOnClient() {
     return this.cqService.getCqServiceVsdStats().getNumCqsOnClient();
   }
 
   /**
    * Returns the number of CQs (active + suspended) on the given region.
-   * 
-   * @param regionName
    */
+  @Override
   public long numCqsOnRegion(String regionName) {
-
     DefaultQueryService queryService =
-        (DefaultQueryService) ((GemFireCacheImpl) CacheFactory.getAnyInstance())
-            .getLocalQueryService();
+        (DefaultQueryService) cqService.getInternalCache().getLocalQueryService();
     try {
       CqQuery[] cqs = queryService.getCqs(regionName);
 


[12/13] geode git commit: Safe refactorings

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index 56243e1..010ec43 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -12,12 +12,69 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.internal.cache;
 
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.Reader;
+import java.io.StringBufferInputStream;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.naming.Context;
+import javax.transaction.TransactionManager;
+
 import com.sun.jna.Native;
 import com.sun.jna.Platform;
 import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.CancelException;
 import org.apache.geode.ForcedDisconnectException;
@@ -89,6 +146,7 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.CacheTime;
+import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionAdvisee;
 import org.apache.geode.distributed.internal.DistributionAdvisor;
@@ -103,7 +161,6 @@ import org.apache.geode.distributed.internal.ReplyProcessor21;
 import org.apache.geode.distributed.internal.ResourceEvent;
 import org.apache.geode.distributed.internal.ResourceEventsListener;
 import org.apache.geode.distributed.internal.ServerLocation;
-import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.locks.DLockService;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.i18n.LogWriterI18n;
@@ -125,7 +182,6 @@ import org.apache.geode.internal.cache.partitioned.RedundancyAlreadyMetException
 import org.apache.geode.internal.cache.persistence.BackupManager;
 import org.apache.geode.internal.cache.persistence.PersistentMemberID;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
-import org.apache.geode.internal.cache.persistence.query.TemporaryResultSetFactory;
 import org.apache.geode.internal.cache.snapshot.CacheSnapshotServiceImpl;
 import org.apache.geode.internal.cache.tier.sockets.AcceptorImpl;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier;
@@ -173,61 +229,6 @@ import org.apache.geode.pdx.internal.PdxInstanceFactoryImpl;
 import org.apache.geode.pdx.internal.PdxInstanceImpl;
 import org.apache.geode.pdx.internal.TypeRegistry;
 import org.apache.geode.redis.GeodeRedisServer;
-import org.apache.logging.log4j.Logger;
-
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.io.PrintStream;
-import java.io.Reader;
-import java.io.StringBufferInputStream;
-import java.io.StringWriter;
-import java.io.Writer;
-import java.net.InetSocketAddress;
-import java.net.URL;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.ServiceLoader;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import javax.naming.Context;
 
 // TODO: somebody Come up with more reasonable values for {@link #DEFAULT_LOCK_TIMEOUT}, etc.
 /**
@@ -238,23 +239,22 @@ public class GemFireCacheImpl
     implements InternalCache, ClientCache, HasCachePerfStats, DistributionAdvisee, CacheTime {
   private static final Logger logger = LogService.getLogger();
 
-  // moved *SERIAL_NUMBER stuff to DistributionAdvisor
-
   /** The default number of seconds to wait for a distributed lock */
-  public static final int DEFAULT_LOCK_TIMEOUT = Integer
-      .getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.defaultLockTimeout", 60).intValue();
+  public static final int DEFAULT_LOCK_TIMEOUT =
+      Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.defaultLockTimeout", 60);
 
   /**
    * The default duration (in seconds) of a lease on a distributed lock
    */
-  public static final int DEFAULT_LOCK_LEASE = Integer
-      .getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.defaultLockLease", 120).intValue();
+  public static final int DEFAULT_LOCK_LEASE =
+      Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.defaultLockLease", 120);
 
   /** The default "copy on read" attribute value */
   public static final boolean DEFAULT_COPY_ON_READ = false;
 
   /** the last instance of GemFireCache created */
   private static volatile GemFireCacheImpl instance = null;
+
   /**
    * Just like instance but is valid for a bit longer so that pdx can still find the cache during a
    * close.
@@ -262,16 +262,15 @@ public class GemFireCacheImpl
   private static volatile GemFireCacheImpl pdxInstance = null;
 
   /**
-   * The default amount of time to wait for a <code>netSearch</code> to complete
+   * The default amount of time to wait for a {@code netSearch} to complete
    */
-  public static final int DEFAULT_SEARCH_TIMEOUT = Integer
-      .getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.defaultSearchTimeout", 300).intValue();
+  public static final int DEFAULT_SEARCH_TIMEOUT =
+      Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.defaultSearchTimeout", 300);
 
   /**
-   * The <code>CacheLifecycleListener</code> s that have been registered in this VM
+   * The {@code CacheLifecycleListener} s that have been registered in this VM
    */
-  private static final Set<CacheLifecycleListener> cacheLifecycleListeners =
-      new HashSet<CacheLifecycleListener>();
+  private static final Set<CacheLifecycleListener> cacheLifecycleListeners = new HashSet<>();
 
   /**
    * Define gemfire.Cache.ASYNC_EVENT_LISTENERS=true to invoke event listeners in the background
@@ -284,80 +283,63 @@ public class GemFireCacheImpl
    * (the default) then the size of the entry value is unchanged by a delta application. Not a final
    * so that tests can change this value.
    *
-   * @since GemFire hitachi 6.1.2.9
+   * @since GemFire h****** 6.1.2.9
    */
-  public static boolean DELTAS_RECALCULATE_SIZE =
+  static boolean DELTAS_RECALCULATE_SIZE =
       Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "DELTAS_RECALCULATE_SIZE");
 
-  public static final int EVENT_QUEUE_LIMIT = Integer
-      .getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.EVENT_QUEUE_LIMIT", 4096).intValue();
-  public static final int EVENT_THREAD_LIMIT = Integer
-      .getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.EVENT_THREAD_LIMIT", 16).intValue();
+  private static final int EVENT_QUEUE_LIMIT =
+      Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.EVENT_QUEUE_LIMIT", 4096);
+
+  static final int EVENT_THREAD_LIMIT =
+      Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.EVENT_THREAD_LIMIT", 16);
 
   /**
    * System property to limit the max query-execution time. By default its turned off (-1), the time
-   * is set in MiliSecs.
+   * is set in milliseconds.
    */
   public static final int MAX_QUERY_EXECUTION_TIME =
-      Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.MAX_QUERY_EXECUTION_TIME", -1)
-          .intValue();
+      Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "Cache.MAX_QUERY_EXECUTION_TIME", -1);
 
   /**
    * System property to disable query monitor even if resource manager is in use
    */
-  public final boolean QUERY_MONITOR_DISABLED_FOR_LOW_MEM = Boolean
+  private final boolean queryMonitorDisabledForLowMem = Boolean
       .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "Cache.DISABLE_QUERY_MONITOR_FOR_LOW_MEMORY");
 
   /**
    * Property set to true if resource manager heap percentage is set and query monitor is required
    */
-  public static Boolean QUERY_MONITOR_REQUIRED_FOR_RESOURCE_MANAGER = Boolean.FALSE;
-
-  /**
-   * This property defines internal function that will get executed on each node to fetch active
-   * REST service endpoints (servers).
-   */
-  public static final String FIND_REST_ENABLED_SERVERS_FUNCTION_ID =
-      FindRestEnabledServersFunction.class.getName();
+  private static boolean queryMonitorRequiredForResourceManager = false;
 
-  /**
-   * True if the user is allowed lock when memory resources appear to be overcommitted.
-   */
-  public static final boolean ALLOW_MEMORY_LOCK_WHEN_OVERCOMMITTED =
-      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "Cache.ALLOW_MEMORY_OVERCOMMIT");
-
-  // time in ms
+  /** time in milliseconds */
   private static final int FIVE_HOURS = 5 * 60 * 60 * 1000;
-  /** To test MAX_QUERY_EXECUTION_TIME option. */
-  public int TEST_MAX_QUERY_EXECUTION_TIME = -1;
-  public boolean TEST_MAX_QUERY_EXECUTION_TIME_OVERRIDE_EXCEPTION = false;
 
-  // ///////////////////// Instance Fields ///////////////////////
+  /** To test MAX_QUERY_EXECUTION_TIME option. */
+  public int testMaxQueryExecutionTime = -1;
 
   private final InternalDistributedSystem system;
 
   private final DM dm;
 
-  // This is a HashMap because I know that clear() on it does
-  // not allocate objects.
-  private final HashMap rootRegions;
+  private final Map<String, LocalRegion> rootRegions;
 
   /**
    * True if this cache is being created by a ClientCacheFactory.
    */
   private final boolean isClient;
-  private PoolFactory clientpf;
+
+  private PoolFactory poolFactory;
+
   /**
    * It is not final to allow cache.xml parsing to set it.
    */
   private Pool defaultPool;
 
-  private final ConcurrentMap pathToRegion = new ConcurrentHashMap();
+  private final ConcurrentMap<String, Region<?, ?>> pathToRegion = new ConcurrentHashMap<>();
 
-  protected volatile boolean isInitialized = false;
-  protected volatile boolean isClosing = false;
-  protected volatile boolean closingGatewaySendersByShutdownAll = false;
-  protected volatile boolean closingGatewayReceiversByShutdownAll = false;
+  private volatile boolean isInitialized = false;
+  volatile boolean isClosing = false;
 
   /** Amount of time (in seconds) to wait for a distributed lock */
   private int lockTimeout = DEFAULT_LOCK_TIMEOUT;
@@ -365,7 +347,7 @@ public class GemFireCacheImpl
   /** Amount of time a lease of a distributed lock lasts */
   private int lockLease = DEFAULT_LOCK_LEASE;
 
-  /** Amount of time to wait for a <code>netSearch</code> to complete */
+  /** Amount of time to wait for a {@code netSearch} to complete */
   private int searchTimeout = DEFAULT_SEARCH_TIMEOUT;
 
   private final CachePerfStats cachePerfStats;
@@ -381,14 +363,14 @@ public class GemFireCacheImpl
    * retrieval operations. It is assumed that the traversal operations on cache servers list vastly
    * outnumber the mutative operations such as add, remove.
    */
-  private volatile List allCacheServers = new CopyOnWriteArrayList();
+  private final List<CacheServerImpl> allCacheServers = new CopyOnWriteArrayList<>();
 
   /**
    * Controls updates to the list of all gateway senders
    *
    * @see #allGatewaySenders
    */
-  public final Object allGatewaySendersLock = new Object();
+  private final Object allGatewaySendersLock = new Object();
 
   /**
    * the set of all gateway senders. It may be fetched safely (for enumeration), but updates must by
@@ -400,22 +382,20 @@ public class GemFireCacheImpl
    * The list of all async event queues added to the cache. CopyOnWriteArrayList is used to allow
    * concurrent add, remove and retrieval operations.
    */
-  private volatile Set<AsyncEventQueue> allVisibleAsyncEventQueues =
-      new CopyOnWriteArraySet<AsyncEventQueue>();
+  private final Set<AsyncEventQueue> allVisibleAsyncEventQueues = new CopyOnWriteArraySet<>();
 
   /**
    * The list of all async event queues added to the cache. CopyOnWriteArrayList is used to allow
    * concurrent add, remove and retrieval operations.
    */
-  private volatile Set<AsyncEventQueue> allAsyncEventQueues =
-      new CopyOnWriteArraySet<AsyncEventQueue>();
+  private final Set<AsyncEventQueue> allAsyncEventQueues = new CopyOnWriteArraySet<>();
 
   /**
    * Controls updates to the list of all gateway receivers
    *
    * @see #allGatewayReceivers
    */
-  public final Object allGatewayReceiversLock = new Object();
+  private final Object allGatewayReceiversLock = new Object();
 
   /**
    * the list of all gateway Receivers. It may be fetched safely (for enumeration), but updates must
@@ -423,10 +403,10 @@ public class GemFireCacheImpl
    */
   private volatile Set<GatewayReceiver> allGatewayReceivers = Collections.emptySet();
 
-  /** PartitionedRegion instances (for required-events notification */
-  // This is a HashSet because I know that clear() on it does not
-  // allocate any objects.
-  private final HashSet<PartitionedRegion> partitionedRegions = new HashSet<PartitionedRegion>();
+  /**
+   * PartitionedRegion instances (for required-events notification
+   */
+  private final Set<PartitionedRegion> partitionedRegions = new HashSet<>();
 
   /**
    * Fix for 42051 This is a map of regions that are in the process of being destroyed. We could
@@ -436,14 +416,14 @@ public class GemFireCacheImpl
    * that ID if it receives it as part of the persistent view.
    */
   private final ConcurrentMap<String, DistributedRegion> regionsInDestroy =
-      new ConcurrentHashMap<String, DistributedRegion>();
+      new ConcurrentHashMap<>();
 
-  public final Object allGatewayHubsLock = new Object();
+  private final Object allGatewayHubsLock = new Object();
 
   /**
    * conflict resolver for WAN, if any
    * 
-   * @guarded.By {@link #allGatewayHubsLock}
+   * GuardedBy {@link #allGatewayHubsLock}
    */
   private GatewayConflictResolver gatewayConflictResolver;
 
@@ -451,7 +431,7 @@ public class GemFireCacheImpl
   private boolean isServer = false;
 
   /** transaction manager for this cache */
-  private final TXManagerImpl txMgr;
+  private final TXManagerImpl transactionManager;
 
   private RestAgent restAgent;
 
@@ -461,7 +441,8 @@ public class GemFireCacheImpl
   private volatile boolean copyOnRead = DEFAULT_COPY_ON_READ;
 
   /** The named region attributes registered with this cache. */
-  private final Map namedRegionAttributes = Collections.synchronizedMap(new HashMap());
+  private final Map<String, RegionAttributes<?, ?>> namedRegionAttributes =
+      Collections.synchronizedMap(new HashMap<>());
 
   /**
    * if this cache was forced to close due to a forced-disconnect, we retain a
@@ -473,24 +454,24 @@ public class GemFireCacheImpl
    * if this cache was forced to close due to a forced-disconnect or system failure, this keeps
    * track of the reason
    */
-  protected volatile Throwable disconnectCause = null;
+  volatile Throwable disconnectCause = null;
 
   /** context where this cache was created -- for debugging, really... */
-  public Exception creationStack = null;
+  private Exception creationStack = null;
 
   /**
    * a system timer task for cleaning up old bridge thread event entries
    */
-  private EventTracker.ExpiryTask recordedEventSweeper;
+  private final EventTracker.ExpiryTask recordedEventSweeper;
 
-  private TombstoneService tombstoneService;
+  private final TombstoneService tombstoneService;
 
   /**
    * DistributedLockService for PartitionedRegions. Remains null until the first PartitionedRegion
    * is created. Destroyed by GemFireCache when closing the cache. Protected by synchronization on
    * this GemFireCache.
    *
-   * @guarded.By prLockServiceLock
+   * GuardedBy prLockServiceLock
    */
   private DistributedLockService prLockService;
 
@@ -503,7 +484,7 @@ public class GemFireCacheImpl
    * DistributedLockService for GatewaySenders. Remains null until the first GatewaySender is
    * created. Destroyed by GemFireCache when closing the cache.
    * 
-   * @guarded.By gatewayLockServiceLock
+   * GuardedBy gatewayLockServiceLock
    */
   private volatile DistributedLockService gatewayLockService;
 
@@ -514,7 +495,7 @@ public class GemFireCacheImpl
 
   private final InternalResourceManager resourceManager;
 
-  private final AtomicReference<BackupManager> backupManager = new AtomicReference<BackupManager>();
+  private final AtomicReference<BackupManager> backupManager = new AtomicReference<>();
 
   private HeapEvictor heapEvictor = null;
 
@@ -524,7 +505,7 @@ public class GemFireCacheImpl
 
   private final Object offHeapEvictorLock = new Object();
 
-  private ResourceEventsListener listener;
+  private ResourceEventsListener resourceEventsListener;
 
   /**
    * Enabled when CacheExistsException issues arise in debugging
@@ -539,7 +520,7 @@ public class GemFireCacheImpl
 
   private final PersistentMemberManager persistentMemberManager;
 
-  private ClientMetadataService clientMetadatService = null;
+  private ClientMetadataService clientMetadataService = null;
 
   private final Object clientMetaDatServiceLock = new Object();
 
@@ -557,20 +538,17 @@ public class GemFireCacheImpl
 
   private final DiskStoreMonitor diskMonitor;
 
-  // Stores the properties used to initialize declarables.
-  private final Map<Declarable, Properties> declarablePropertiesMap =
-      new ConcurrentHashMap<Declarable, Properties>();
+  /**
+   * Stores the properties used to initialize declarables.
+   */
+  private final Map<Declarable, Properties> declarablePropertiesMap = new ConcurrentHashMap<>();
 
   /** {@link PropertyResolver} to resolve ${} type property strings */
-  protected static PropertyResolver resolver;
+  private final PropertyResolver resolver;
 
-  protected static boolean xmlParameterizationEnabled =
+  private static final boolean XML_PARAMETERIZATION_ENABLED =
       !Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "xml.parameterization.disabled");
 
-  public static Runnable internalBeforeApplyChanges;
-
-  public static Runnable internalBeforeNonTXBasicPut;
-
   /**
    * the memcachedServer instance that is started when {@link DistributionConfig#getMemcachedPort()}
    * is specified
@@ -587,23 +565,19 @@ public class GemFireCacheImpl
    * 
    * @since GemFire 8.1
    */
-  private SimpleExtensionPoint<Cache> extensionPoint = new SimpleExtensionPoint<Cache>(this, this);
+  private final SimpleExtensionPoint<Cache> extensionPoint = new SimpleExtensionPoint<>(this, this);
 
   private final CqService cqService;
 
-  private final Set<RegionListener> regionListeners = new ConcurrentHashSet<RegionListener>();
+  private final Set<RegionListener> regionListeners = new ConcurrentHashSet<>();
 
-  private final Map<Class<? extends CacheService>, CacheService> services =
-      new HashMap<Class<? extends CacheService>, CacheService>();
+  private final Map<Class<? extends CacheService>, CacheService> services = new HashMap<>();
 
   public static final int DEFAULT_CLIENT_FUNCTION_TIMEOUT = 0;
 
   private static int clientFunctionTimeout;
 
-  private final static Boolean DISABLE_AUTO_EVICTION =
-      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "disableAutoEviction");
-
-  private static SecurityService securityService = SecurityService.getSecurityService();
+  private final SecurityService securityService = SecurityService.getSecurityService();
 
   static {
     // this works around jdk bug 6427854, reported in ticket #44434
@@ -629,15 +603,13 @@ public class GemFireCacheImpl
    *        and stack as well as new memory mapped files or shared memory regions.
    * 
    * @return 0 if success, non-zero if error and errno set
-   * 
    */
   private static native int mlockall(int flags);
 
   public static void lockMemory() {
-    int result = 0;
     try {
       Native.register(Platform.C_LIBRARY_NAME);
-      result = mlockall(1);
+      int result = mlockall(1);
       if (result == 0) {
         return;
       }
@@ -645,14 +617,14 @@ public class GemFireCacheImpl
       throw new IllegalStateException("Error trying to lock memory", t);
     }
 
-    int errno = Native.getLastError();
-    String msg = "mlockall failed: " + errno;
-    if (errno == 1 || errno == 12) { // EPERM || ENOMEM
-      msg = "Unable to lock memory due to insufficient free space or privileges.  "
+    int lastError = Native.getLastError();
+    String message = "mlockall failed: " + lastError;
+    if (lastError == 1 || lastError == 12) { // EPERM || ENOMEM
+      message = "Unable to lock memory due to insufficient free space or privileges.  "
           + "Please check the RLIMIT_MEMLOCK soft resource limit (ulimit -l) and "
           + "increase the available memory if needed";
     }
-    throw new IllegalStateException(msg);
+    throw new IllegalStateException(message);
   }
 
   /**
@@ -660,19 +632,19 @@ public class GemFireCacheImpl
    * {@link org.apache.geode.cache.CacheExistsException})
    */
   @Override
-  public String toString() {
-    final StringBuffer sb = new StringBuffer();
+  public final String toString() {
+    final StringBuilder sb = new StringBuilder();
     sb.append("GemFireCache[");
-    sb.append("id = " + System.identityHashCode(this));
-    sb.append("; isClosing = " + this.isClosing);
-    sb.append("; isShutDownAll = " + isCacheAtShutdownAll());
-    sb.append("; created = " + this.creationDate);
-    sb.append("; server = " + this.isServer);
-    sb.append("; copyOnRead = " + this.copyOnRead);
-    sb.append("; lockLease = " + this.lockLease);
-    sb.append("; lockTimeout = " + this.lockTimeout);
+    sb.append("id = ").append(System.identityHashCode(this));
+    sb.append("; isClosing = ").append(this.isClosing);
+    sb.append("; isShutDownAll = ").append(isCacheAtShutdownAll());
+    sb.append("; created = ").append(this.creationDate);
+    sb.append("; server = ").append(this.isServer);
+    sb.append("; copyOnRead = ").append(this.copyOnRead);
+    sb.append("; lockLease = ").append(this.lockLease);
+    sb.append("; lockTimeout = ").append(this.lockTimeout);
     if (this.creationStack != null) {
-      sb.append("\nCreation context:\n");
+      sb.append(System.lineSeparator()).append("Creation context:").append(System.lineSeparator());
       OutputStream os = new OutputStream() {
         @Override
         public void write(int i) {
@@ -694,7 +666,7 @@ public class GemFireCacheImpl
     return instance;
   }
 
-  /*
+  /**
    * Used for testing, retain the old instance in the test and re-set the value when test completes
    */
   public static GemFireCacheImpl setInstanceForTests(GemFireCacheImpl cache) {
@@ -709,7 +681,7 @@ public class GemFireCacheImpl
    * @return the existing cache
    * @throws CacheClosedException if an existing cache can not be found.
    */
-  public static final GemFireCacheImpl getExisting() {
+  public static GemFireCacheImpl getExisting() {
     final GemFireCacheImpl result = instance;
     if (result != null && !result.isClosing) {
       return result;
@@ -748,15 +720,6 @@ public class GemFireCacheImpl
     return result;
   }
 
-  // /**
-  // * @deprecated remove when Lise allows a Hydra VM to
-  // * be re-created
-  // */
-  // public static void clearInstance() {
-  // System.err.println("DEBUG: do not commit GemFireCache#clearInstance");
-  // instance = null;
-  // }
-
   public static GemFireCacheImpl createClient(DistributedSystem system, PoolFactory pf,
       CacheConfig cacheConfig) {
     return basicCreate(system, true, cacheConfig, pf, true, ASYNC_EVENT_LISTENERS, null);
@@ -766,7 +729,7 @@ public class GemFireCacheImpl
     return basicCreate(system, true, cacheConfig, null, false, ASYNC_EVENT_LISTENERS, null);
   }
 
-  public static GemFireCacheImpl createWithAsyncEventListeners(DistributedSystem system,
+  static GemFireCacheImpl createWithAsyncEventListeners(DistributedSystem system,
       CacheConfig cacheConfig, TypeRegistry typeRegistry) {
     return basicCreate(system, true, cacheConfig, null, false, true, typeRegistry);
   }
@@ -776,8 +739,6 @@ public class GemFireCacheImpl
     return basicCreate(system, existingOk, cacheConfig, null, false, ASYNC_EVENT_LISTENERS, null);
   }
 
-
-
   private static GemFireCacheImpl basicCreate(DistributedSystem system, boolean existingOk,
       CacheConfig cacheConfig, PoolFactory pf, boolean isClient, boolean asyncEventListeners,
       TypeRegistry typeRegistry) throws CacheExistsException, TimeoutException,
@@ -793,7 +754,7 @@ public class GemFireCacheImpl
         return instance;
       }
     } catch (CacheXmlException | IllegalArgumentException e) {
-      logger.error(e.getLocalizedMessage());
+      logger.error(e.getLocalizedMessage()); // TODO: log the full stack trace or not?
       throw e;
     } catch (Error | RuntimeException e) {
       logger.error(e);
@@ -821,15 +782,15 @@ public class GemFireCacheImpl
   }
 
   /**
-   * Creates a new instance of GemFireCache and populates it according to the
-   * <code>cache.xml</code>, if appropriate.
+   * Creates a new instance of GemFireCache and populates it according to the {@code cache.xml}, if
+   * appropriate.
    * 
    * @param typeRegistry: currently only unit tests set this parameter to a non-null value
    */
   private GemFireCacheImpl(boolean isClient, PoolFactory pf, DistributedSystem system,
       CacheConfig cacheConfig, boolean asyncEventListeners, TypeRegistry typeRegistry) {
     this.isClient = isClient;
-    this.clientpf = pf;
+    this.poolFactory = pf;
     this.cacheConfig = cacheConfig; // do early for bug 43213
     this.pdxRegistry = typeRegistry;
 
@@ -846,28 +807,25 @@ public class GemFireCacheImpl
         // We only support management on members of a distributed system
         // Should do this: if (!getSystem().isLoner()) {
         // but it causes quickstart.CqClientTest to hang
-        this.listener = new ManagementListener();
-        this.system.addResourceListener(listener);
+        this.resourceEventsListener = new ManagementListener();
+        this.system.addResourceListener(this.resourceEventsListener);
         if (this.system.isLoner()) {
           this.system.getInternalLogWriter()
               .info(LocalizedStrings.GemFireCacheImpl_RUNNING_IN_LOCAL_MODE);
         }
       } else {
-        getLogger().info("Running in client mode");
-        this.listener = null;
+        logger.info("Running in client mode");
+        this.resourceEventsListener = null;
       }
 
       // Don't let admin-only VMs create Cache's just yet.
-      DM dm = this.system.getDistributionManager();
-      if (dm instanceof DistributionManager) {
-        if (((DistributionManager) dm).getDMType() == DistributionManager.ADMIN_ONLY_DM_TYPE) {
-          throw new IllegalStateException(
-              LocalizedStrings.GemFireCache_CANNOT_CREATE_A_CACHE_IN_AN_ADMINONLY_VM
-                  .toLocalizedString());
-        }
+      if (this.dm.getDMType() == DistributionManager.ADMIN_ONLY_DM_TYPE) {
+        throw new IllegalStateException(
+            LocalizedStrings.GemFireCache_CANNOT_CREATE_A_CACHE_IN_AN_ADMINONLY_VM
+                .toLocalizedString());
       }
 
-      this.rootRegions = new HashMap();
+      this.rootRegions = new HashMap<>();
 
       this.cqService = CqServiceProvider.create(this);
 
@@ -875,44 +833,39 @@ public class GemFireCacheImpl
       this.cachePerfStats = new CachePerfStats(system);
       CachePerfStats.enableClockStats = this.system.getConfig().getEnableTimeStatistics();
 
-      this.txMgr = new TXManagerImpl(this.cachePerfStats, this);
-      dm.addMembershipListener(this.txMgr);
+      this.transactionManager = new TXManagerImpl(this.cachePerfStats, this);
+      this.dm.addMembershipListener(this.transactionManager);
 
       this.creationDate = new Date();
 
       this.persistentMemberManager = new PersistentMemberManager();
 
       if (asyncEventListeners) {
-        final ThreadGroup group =
+        final ThreadGroup threadGroup =
             LoggingThreadGroup.createThreadGroup("Message Event Threads", logger);
-        ThreadFactory tf = new ThreadFactory() {
-          @Override
-          public Thread newThread(final Runnable command) {
-            final Runnable r = new Runnable() {
-              @Override
-              public void run() {
-                ConnectionTable.threadWantsSharedResources();
-                command.run();
-              }
-            };
-            Thread thread = new Thread(group, r, "Message Event Thread");
-            thread.setDaemon(true);
-            return thread;
-          }
+        ThreadFactory threadFactory = (Runnable command) -> {
+          final Runnable runnable = () -> {
+            ConnectionTable.threadWantsSharedResources();
+            command.run();
+          };
+          Thread thread = new Thread(threadGroup, runnable, "Message Event Thread");
+          thread.setDaemon(true);
+          return thread;
         };
-        ArrayBlockingQueue q = new ArrayBlockingQueue(EVENT_QUEUE_LIMIT);
-        this.eventThreadPool = new PooledExecutorWithDMStats(q, EVENT_THREAD_LIMIT,
-            this.cachePerfStats.getEventPoolHelper(), tf, 1000);
+        ArrayBlockingQueue<Runnable> queue = new ArrayBlockingQueue<>(EVENT_QUEUE_LIMIT);
+        this.eventThreadPool = new PooledExecutorWithDMStats(queue, EVENT_THREAD_LIMIT,
+            this.cachePerfStats.getEventPoolHelper(), threadFactory, 1000);
       } else {
         this.eventThreadPool = null;
       }
 
       // Initialize the advisor here, but wait to exchange profiles until cache is fully built
       this.resourceAdvisor = ResourceAdvisor.createResourceAdvisor(this);
+
       // Initialize the advisor here, but wait to exchange profiles until cache is fully built
       this.jmxAdvisor = JmxManagerAdvisor.createJmxManagerAdvisor(new JmxManagerAdvisee(this));
 
-      resourceManager = InternalResourceManager.createResourceManager(this);
+      this.resourceManager = InternalResourceManager.createResourceManager(this);
       this.serialNumber = DistributionAdvisor.createSerialNumber();
 
       getInternalResourceManager().addResourceListener(ResourceType.HEAP_MEMORY, getHeapEvictor());
@@ -925,15 +878,15 @@ public class GemFireCacheImpl
             getOffHeapEvictor());
       }
 
-      recordedEventSweeper = EventTracker.startTrackerServices(this);
-      tombstoneService = TombstoneService.initialize(this);
+      this.recordedEventSweeper = EventTracker.startTrackerServices(this);
+      this.tombstoneService = TombstoneService.initialize(this);
 
       TypeRegistry.init();
       basicSetPdxSerializer(this.cacheConfig.getPdxSerializer());
       TypeRegistry.open();
 
       if (!isClient()) {
-        // Initialize the QRM thread freqeuncy to default (1 second )to prevent spill
+        // Initialize the QRM thread frequency to default (1 second )to prevent spill
         // over from previous Cache , as the interval is stored in a static
         // volatile field.
         HARegionQueue.setMessageSyncInterval(HARegionQueue.DEFAULT_MESSAGE_SYNC_INTERVAL);
@@ -948,26 +901,28 @@ public class GemFireCacheImpl
       }
 
       this.txEntryStateFactory = TXEntryState.getFactory();
-      if (xmlParameterizationEnabled) {
-        /** If product properties file is available replace properties from there */
+      if (XML_PARAMETERIZATION_ENABLED) {
+        // If product properties file is available replace properties from there
         Properties userProps = this.system.getConfig().getUserDefinedProps();
         if (userProps != null && !userProps.isEmpty()) {
-          resolver = new CacheXmlPropertyResolver(false,
+          this.resolver = new CacheXmlPropertyResolver(false,
               PropertyResolver.NO_SYSTEM_PROPERTIES_OVERRIDE, userProps);
         } else {
-          resolver = new CacheXmlPropertyResolver(false,
+          this.resolver = new CacheXmlPropertyResolver(false,
               PropertyResolver.NO_SYSTEM_PROPERTIES_OVERRIDE, null);
         }
+      } else {
+        this.resolver = null;
       }
 
       SystemFailure.signalCacheCreate();
 
-      diskMonitor = new DiskStoreMonitor();
+      this.diskMonitor = new DiskStoreMonitor();
     } // synchronized
   }
 
   public boolean isRESTServiceRunning() {
-    return isRESTServiceRunning;
+    return this.isRESTServiceRunning;
   }
 
   public void setRESTServiceRunning(boolean isRESTServiceRunning) {
@@ -980,23 +935,25 @@ public class GemFireCacheImpl
    * @return RestAgent
    */
   public RestAgent getRestAgent() {
-    return restAgent;
+    return this.restAgent;
   }
 
-  /*****
+  /**
    * Request the shared configuration from the locator(s) which have the Cluster config service
    * running
    */
-  public ConfigurationResponse requestSharedConfiguration() {
+  private ConfigurationResponse requestSharedConfiguration() {
     final DistributionConfig config = this.system.getConfig();
 
-    if (!(dm instanceof DistributionManager))
+    if (!(this.dm instanceof DistributionManager)) {
       return null;
+    }
 
     // do nothing if this vm is/has locator or this is a client
-    if (((DistributionManager) dm).getDMType() == DistributionManager.LOCATOR_DM_TYPE || isClient
-        || Locator.getLocator() != null)
+    if (this.dm.getDMType() == DistributionManager.LOCATOR_DM_TYPE || this.isClient
+        || Locator.getLocator() != null) {
       return null;
+    }
 
     // can't simply return null if server is not using shared configuration, since we need to find
     // out
@@ -1015,13 +972,11 @@ public class GemFireCacheImpl
       return null;
     }
 
-
-    ConfigurationResponse response = null;
     List<String> locatorConnectionStrings = getSharedConfigLocatorConnectionStringList();
 
     try {
-      response = ClusterConfigurationLoader.requestConfigurationFromLocators(system.getConfig(),
-          locatorConnectionStrings);
+      ConfigurationResponse response = ClusterConfigurationLoader
+          .requestConfigurationFromLocators(this.system.getConfig(), locatorConnectionStrings);
 
       // log the configuration received from the locator
       logger.info(LocalizedMessage
@@ -1031,7 +986,7 @@ public class GemFireCacheImpl
       Configuration clusterConfig =
           response.getRequestedConfiguration().get(ClusterConfigurationService.CLUSTER_CONFIG);
       Properties clusterSecProperties =
-          (clusterConfig == null) ? new Properties() : clusterConfig.getGemfireProperties();
+          clusterConfig == null ? new Properties() : clusterConfig.getGemfireProperties();
 
       // If not using shared configuration, return null or throw an exception is locator is secured
       if (!config.getUseSharedConfiguration()) {
@@ -1064,15 +1019,10 @@ public class GemFireCacheImpl
     }
   }
 
-  public void deployJarsRecevedFromClusterConfiguration(ConfigurationResponse response) {
+  private void deployJarsReceivedFromClusterConfiguration(ConfigurationResponse response) {
     try {
       ClusterConfigurationLoader.deployJarsReceivedFromClusterConfiguration(this, response);
-    } catch (IOException e) {
-      throw new GemFireConfigException(
-          LocalizedStrings.GemFireCache_EXCEPTION_OCCURED_WHILE_DEPLOYING_JARS_FROM_SHARED_CONDFIGURATION
-              .toLocalizedString(),
-          e);
-    } catch (ClassNotFoundException e) {
+    } catch (IOException | ClassNotFoundException e) {
       throw new GemFireConfigException(
           LocalizedStrings.GemFireCache_EXCEPTION_OCCURED_WHILE_DEPLOYING_JARS_FROM_SHARED_CONDFIGURATION
               .toLocalizedString(),
@@ -1080,10 +1030,10 @@ public class GemFireCacheImpl
     }
   }
 
-
-  // When called, clusterProps and serverProps and key could not be null
-  public static boolean isMisConfigured(Properties clusterProps, Properties serverProps,
-      String key) {
+  /**
+   * When called, clusterProps and serverProps and key could not be null
+   */
+  static boolean isMisConfigured(Properties clusterProps, Properties serverProps, String key) {
     String clusterPropValue = clusterProps.getProperty(key);
     String serverPropValue = serverProps.getProperty(key);
 
@@ -1095,36 +1045,31 @@ public class GemFireCacheImpl
     if (StringUtils.isBlank(clusterPropValue))
       return true;
 
-    // at this point check for eqality
+    // at this point check for equality
     return !clusterPropValue.equals(serverPropValue);
   }
 
-  public List<String> getSharedConfigLocatorConnectionStringList() {
-    List<String> locatorConnectionStringList = new ArrayList<String>();
+  private List<String> getSharedConfigLocatorConnectionStringList() {
+    List<String> locatorConnectionStringList = new ArrayList<>();
 
-    Map<InternalDistributedMember, Collection<String>> scl =
+    Map<InternalDistributedMember, Collection<String>> locatorsWithClusterConfig =
         this.getDistributionManager().getAllHostedLocatorsWithSharedConfiguration();
 
     // If there are no locators with Shared configuration, that means the system has been started
     // without shared configuration
     // then do not make requests to the locators
-    if (!scl.isEmpty()) {
-      Set<Entry<InternalDistributedMember, Collection<String>>> locs = scl.entrySet();
+    if (!locatorsWithClusterConfig.isEmpty()) {
+      Set<Entry<InternalDistributedMember, Collection<String>>> locators =
+          locatorsWithClusterConfig.entrySet();
 
-      for (Entry<InternalDistributedMember, Collection<String>> loc : locs) {
+      for (Entry<InternalDistributedMember, Collection<String>> loc : locators) {
         Collection<String> locStrings = loc.getValue();
-        Iterator<String> locStringIter = locStrings.iterator();
-
-        while (locStringIter.hasNext()) {
-          locatorConnectionStringList.add(locStringIter.next());
-        }
+        locatorConnectionStringList.addAll(locStrings);
       }
     }
     return locatorConnectionStringList;
   }
 
-
-
   /**
    * Used by unit tests to force cache creation to use a test generated cache.xml
    */
@@ -1135,7 +1080,7 @@ public class GemFireCacheImpl
    * @see #hasPool()
    */
   @Override
-  public boolean isClient() {
+  public final boolean isClient() {
     return this.isClient;
   }
 
@@ -1149,7 +1094,7 @@ public class GemFireCacheImpl
     return this.isClient || !getAllPools().isEmpty();
   }
 
-  private Collection<Pool> getAllPools() {
+  private static Collection<Pool> getAllPools() {
     Collection<Pool> pools = PoolManagerImpl.getPMI().getMap().values();
     for (Iterator<Pool> itr = pools.iterator(); itr.hasNext();) {
       PoolImpl pool = (PoolImpl) itr.next();
@@ -1168,8 +1113,8 @@ public class GemFireCacheImpl
     return this.defaultPool;
   }
 
-  private void setDefaultPool(Pool v) {
-    this.defaultPool = v;
+  private void setDefaultPool(Pool value) {
+    this.defaultPool = value;
   }
 
   /**
@@ -1184,9 +1129,7 @@ public class GemFireCacheImpl
     GemFireCacheImpl.instance = this;
     GemFireCacheImpl.pdxInstance = this;
 
-    for (Iterator<CacheLifecycleListener> iter = cacheLifecycleListeners.iterator(); iter
-        .hasNext();) {
-      CacheLifecycleListener listener = (CacheLifecycleListener) iter.next();
+    for (CacheLifecycleListener listener : cacheLifecycleListeners) {
       listener.cacheCreated(this);
     }
 
@@ -1194,24 +1137,23 @@ public class GemFireCacheImpl
 
     // request and check cluster configuration
     ConfigurationResponse configurationResponse = requestSharedConfiguration();
-    deployJarsRecevedFromClusterConfiguration(configurationResponse);
+    deployJarsReceivedFromClusterConfiguration(configurationResponse);
 
     // apply the cluster's properties configuration and initialize security using that configuration
     ClusterConfigurationLoader.applyClusterPropertiesConfiguration(this, configurationResponse,
-        system.getConfig());
+        this.system.getConfig());
 
     // first initialize the security service using the security properties
-    securityService.initSecurity(system.getConfig().getSecurityProps());
+    this.securityService.initSecurity(this.system.getConfig().getSecurityProps());
     // secondly if cacheConfig has a securityManager, use that instead
-    if (cacheConfig.getSecurityManager() != null) {
-      securityService.setSecurityManager(cacheConfig.getSecurityManager());
+    if (this.cacheConfig.getSecurityManager() != null) {
+      this.securityService.setSecurityManager(this.cacheConfig.getSecurityManager());
     }
     // if cacheConfig has a postProcessor, use that instead
-    if (cacheConfig.getPostProcessor() != null) {
-      securityService.setPostProcessor(cacheConfig.getPostProcessor());
+    if (this.cacheConfig.getPostProcessor() != null) {
+      this.securityService.setPostProcessor(this.cacheConfig.getPostProcessor());
     }
 
-
     SystemMemberCacheEventProcessor.send(this, Operation.CACHE_CREATE);
     this.resourceAdvisor.initializationGate();
 
@@ -1225,23 +1167,21 @@ public class GemFireCacheImpl
     // we will not be ready for all the events that cache.xml
     // processing can deliver (region creation, etc.).
     // This call may need to be moved inside initializeDeclarativeCache.
-    /** Entry to GemFire Management service **/
-    this.jmxAdvisor.initializationGate();
+    this.jmxAdvisor.initializationGate(); // Entry to GemFire Management service
 
     // this starts up the ManagementService, register and federate the internal beans
-    system.handleResourceEvent(ResourceEvent.CACHE_CREATE, this);
-
-    boolean completedCacheXml = false;
+    this.system.handleResourceEvent(ResourceEvent.CACHE_CREATE, this);
 
     initializeServices();
 
+    boolean completedCacheXml = false;
     try {
       if (configurationResponse == null) {
         // Deploy all the jars from the deploy working dir.
         ClassPathLoader.getLatest().getJarDeployer().loadPreviouslyDeployedJarsFromDisk();
       }
       ClusterConfigurationLoader.applyClusterXmlConfiguration(this, configurationResponse,
-          system.getConfig());
+          this.system.getConfig());
       initializeDeclarativeCache();
       completedCacheXml = true;
     } finally {
@@ -1256,7 +1196,7 @@ public class GemFireCacheImpl
       }
     }
 
-    this.clientpf = null;
+    this.poolFactory = null;
 
     startColocatedJmxManagerLocator();
 
@@ -1270,7 +1210,7 @@ public class GemFireCacheImpl
         DEFAULT_CLIENT_FUNCTION_TIMEOUT);
     clientFunctionTimeout = time >= 0 ? time : DEFAULT_CLIENT_FUNCTION_TIMEOUT;
 
-    isInitialized = true;
+    this.isInitialized = true;
   }
 
   /**
@@ -1282,35 +1222,35 @@ public class GemFireCacheImpl
     for (CacheService service : loader) {
       service.init(this);
       this.services.put(service.getInterface(), service);
-      system.handleResourceEvent(ResourceEvent.CACHE_SERVICE_CREATE, service);
+      this.system.handleResourceEvent(ResourceEvent.CACHE_SERVICE_CREATE, service);
     }
   }
 
   private boolean isNotJmxManager() {
-    return (this.system.getConfig().getJmxManagerStart() != true);
+    return !this.system.getConfig().getJmxManagerStart();
   }
 
   private boolean isServerNode() {
-    return (this.system.getDistributedMember().getVmKind() != DistributionManager.LOCATOR_DM_TYPE
+    return this.system.getDistributedMember().getVmKind() != DistributionManager.LOCATOR_DM_TYPE
         && this.system.getDistributedMember().getVmKind() != DistributionManager.ADMIN_ONLY_DM_TYPE
-        && !isClient());
+        && !isClient();
   }
 
   private void startRestAgentServer(GemFireCacheImpl cache) {
     if (this.system.getConfig().getStartDevRestApi() && isNotJmxManager() && isServerNode()) {
       this.restAgent = new RestAgent(this.system.getConfig());
-      restAgent.start(cache);
+      this.restAgent.start(cache);
     } else {
       this.restAgent = null;
     }
   }
 
   private void startMemcachedServer() {
-    int port = system.getConfig().getMemcachedPort();
+    int port = this.system.getConfig().getMemcachedPort();
     if (port != 0) {
-      String protocol = system.getConfig().getMemcachedProtocol();
+      String protocol = this.system.getConfig().getMemcachedProtocol();
       assert protocol != null;
-      String bindAddress = system.getConfig().getMemcachedBindAddress();
+      String bindAddress = this.system.getConfig().getMemcachedBindAddress();
       assert bindAddress != null;
       if (bindAddress.equals(DistributionConfig.DEFAULT_MEMCACHED_BIND_ADDRESS)) {
         logger.info(LocalizedMessage.create(
@@ -1328,9 +1268,9 @@ public class GemFireCacheImpl
   }
 
   private void startRedisServer() {
-    int port = system.getConfig().getRedisPort();
+    int port = this.system.getConfig().getRedisPort();
     if (port != 0) {
-      String bindAddress = system.getConfig().getRedisBindAddress();
+      String bindAddress = this.system.getConfig().getRedisBindAddress();
       assert bindAddress != null;
       if (bindAddress.equals(DistributionConfig.DEFAULT_REDIS_BIND_ADDRESS)) {
         getLoggerI18n().info(
@@ -1346,7 +1286,6 @@ public class GemFireCacheImpl
     }
   }
 
-
   @Override
   public URL getCacheXmlURL() {
     if (this.getMyId().getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
@@ -1356,11 +1295,11 @@ public class GemFireCacheImpl
     if (xmlFile == null) {
       xmlFile = this.system.getConfig().getCacheXmlFile();
     }
-    if ("".equals(xmlFile.getName())) {
+    if (xmlFile.getName().isEmpty()) {
       return null;
     }
 
-    URL url = null;
+    URL url;
     if (!xmlFile.exists() || !xmlFile.isFile()) {
       // do a resource search
       String resource = xmlFile.getPath();
@@ -1372,7 +1311,7 @@ public class GemFireCacheImpl
     } else {
       try {
         url = xmlFile.toURL();
-      } catch (IOException ex) {
+      } catch (MalformedURLException ex) {
         throw new CacheXmlException(
             LocalizedStrings.GemFireCache_COULD_NOT_CONVERT_XML_FILE_0_TO_AN_URL
                 .toLocalizedString(xmlFile),
@@ -1386,7 +1325,7 @@ public class GemFireCacheImpl
           throw new CacheXmlException(
               LocalizedStrings.GemFireCache_DECLARATIVE_CACHE_XML_FILERESOURCE_0_DOES_NOT_EXIST
                   .toLocalizedString(xmlFile));
-        } else /* if (!xmlFile.isFile()) */ {
+        } else {
           throw new CacheXmlException(
               LocalizedStrings.GemFireCache_DECLARATIVE_XML_FILE_0_IS_NOT_A_FILE
                   .toLocalizedString(xmlFile));
@@ -1398,21 +1337,20 @@ public class GemFireCacheImpl
   }
 
   /**
-   * Initializes the contents of this <code>Cache</code> according to the declarative caching XML
-   * file specified by the given <code>DistributedSystem</code>. Note that this operation cannot be
-   * performed in the constructor because creating regions in the cache, etc. uses the cache itself
-   * (which isn't initialized until the constructor returns).
+   * Initializes the contents of this {@code Cache} according to the declarative caching XML file
+   * specified by the given {@code DistributedSystem}. Note that this operation cannot be performed
+   * in the constructor because creating regions in the cache, etc. uses the cache itself (which
+   * isn't initialized until the constructor returns).
    *
    * @throws CacheXmlException If something goes wrong while parsing the declarative caching XML
    *         file.
    * @throws TimeoutException If a {@link org.apache.geode.cache.Region#put(Object, Object)}times
    *         out while initializing the cache.
-   * @throws CacheWriterException If a <code>CacheWriterException</code> is thrown while
-   *         initializing the cache.
-   * @throws RegionExistsException If the declarative caching XML file desribes a region that
-   *         already exists (including the root region).
-   * @throws GatewayException If a <code>GatewayException</code> is thrown while initializing the
+   * @throws CacheWriterException If a {@code CacheWriterException} is thrown while initializing the
    *         cache.
+   * @throws RegionExistsException If the declarative caching XML file describes a region that
+   *         already exists (including the root region).
+   * @throws GatewayException If a {@code GatewayException} is thrown while initializing the cache.
    * 
    * @see #loadCacheXml
    */
@@ -1432,9 +1370,9 @@ public class GemFireCacheImpl
       return; // nothing needs to be done
     }
 
+    InputStream stream = null;
     try {
       logCacheXML(url, cacheXmlDescription);
-      InputStream stream = null;
       if (cacheXmlDescription != null) {
         if (logger.isTraceEnabled()) {
           logger.trace("initializing cache with generated XML: {}", cacheXmlDescription);
@@ -1444,40 +1382,48 @@ public class GemFireCacheImpl
         stream = url.openStream();
       }
       loadCacheXml(stream);
-      try {
-        stream.close();
-      } catch (IOException ignore) {
-      }
+
     } catch (IOException ex) {
       throw new CacheXmlException(
           LocalizedStrings.GemFireCache_WHILE_OPENING_CACHE_XML_0_THE_FOLLOWING_ERROR_OCCURRED_1
-              .toLocalizedString(new Object[] {url.toString(), ex}));
+              .toLocalizedString(url.toString(), ex));
 
     } catch (CacheXmlException ex) {
-      CacheXmlException newEx =
-          new CacheXmlException(LocalizedStrings.GemFireCache_WHILE_READING_CACHE_XML_0_1
-              .toLocalizedString(new Object[] {url, ex.getMessage()}));
-      newEx.setStackTrace(ex.getStackTrace());
-      newEx.initCause(ex.getCause());
-      throw newEx;
+      throw new CacheXmlException(LocalizedStrings.GemFireCache_WHILE_READING_CACHE_XML_0_1
+          .toLocalizedString(url, ex.getMessage()), ex);
+
+    } finally {
+      if (stream != null) {
+        try {
+          stream.close();
+        } catch (IOException ignore) {
+        }
+      }
     }
   }
 
-  private void logCacheXML(URL url, String cacheXmlDescription) {
+  private static void logCacheXML(URL url, String cacheXmlDescription) {
     if (cacheXmlDescription == null) {
       StringBuilder sb = new StringBuilder();
+      BufferedReader br = null;
       try {
-        final String EOLN = System.getProperty("line.separator");
-        BufferedReader br = new BufferedReader(new InputStreamReader(url.openStream()));
-        String l = br.readLine();
-        while (l != null) {
-          if (!l.isEmpty()) {
-            sb.append(EOLN).append(l);
+        final String lineSeparator = System.getProperty("line.separator");
+        br = new BufferedReader(new InputStreamReader(url.openStream()));
+        String line = br.readLine();
+        while (line != null) {
+          if (!line.isEmpty()) {
+            sb.append(lineSeparator).append(line);
           }
-          l = br.readLine();
+          line = br.readLine();
         }
-        br.close();
       } catch (IOException ignore) {
+      } finally {
+        if (br != null) {
+          try {
+            br.close();
+          } catch (IOException ignore) {
+          }
+        }
       }
       logger.info(
           LocalizedMessage.create(LocalizedStrings.GemFireCache_INITIALIZING_CACHE_USING__0__1,
@@ -1516,7 +1462,7 @@ public class GemFireCacheImpl
   }
 
   /**
-   * create diskstore factory with default attributes
+   * create diskStore factory with default attributes
    *
    * @since GemFire prPersistSprint2
    */
@@ -1526,7 +1472,7 @@ public class GemFireCacheImpl
   }
 
   /**
-   * create diskstore factory with predefined attributes
+   * create diskStore factory with predefined attributes
    *
    * @since GemFire prPersistSprint2
    */
@@ -1534,22 +1480,16 @@ public class GemFireCacheImpl
     return new DiskStoreFactoryImpl(this, attrs);
   }
 
-  protected class Stopper extends CancelCriterion {
+  class Stopper extends CancelCriterion {
 
-    /*
-     * (non-Javadoc)
-     *
-     * @see org.apache.geode.CancelCriterion#cancelInProgress()
-     */
     @Override
     public String cancelInProgress() {
-      String reason =
-          GemFireCacheImpl.this.getDistributedSystem().getCancelCriterion().cancelInProgress();
+      String reason = getDistributedSystem().getCancelCriterion().cancelInProgress();
       if (reason != null) {
         return reason;
       }
       if (GemFireCacheImpl.this.disconnectCause != null) {
-        return disconnectCause.getMessage();
+        return GemFireCacheImpl.this.disconnectCause.getMessage();
       }
       if (GemFireCacheImpl.this.isClosing) {
         return "The cache is closed."; // this + ": closed";
@@ -1557,42 +1497,37 @@ public class GemFireCacheImpl
       return null;
     }
 
-    /*
-     * (non-Javadoc)
-     *
-     * @see org.apache.geode.CancelCriterion#generateCancelledException(java.lang.Throwable)
-     */
     @Override
-    public RuntimeException generateCancelledException(Throwable e) {
+    public RuntimeException generateCancelledException(Throwable throwable) {
       String reason = cancelInProgress();
       if (reason == null) {
         return null;
       }
       RuntimeException result =
-          getDistributedSystem().getCancelCriterion().generateCancelledException(e);
+          getDistributedSystem().getCancelCriterion().generateCancelledException(throwable);
       if (result != null) {
         return result;
       }
       if (GemFireCacheImpl.this.disconnectCause == null) {
         // No root cause, specify the one given and be done with it.
-        return new CacheClosedException(reason, e);
+        return new CacheClosedException(reason, throwable);
       }
 
-      if (e == null) {
+      if (throwable == null) {
         // Caller did not specify any root cause, so just use our own.
         return new CacheClosedException(reason, GemFireCacheImpl.this.disconnectCause);
       }
 
       // Attempt to stick rootCause at tail end of the exception chain.
-      Throwable nt = e;
+      Throwable nt = throwable;
       while (nt.getCause() != null) {
         nt = nt.getCause();
       }
       try {
         nt.initCause(GemFireCacheImpl.this.disconnectCause);
-        return new CacheClosedException(reason, e);
+        return new CacheClosedException(reason, throwable);
       } catch (IllegalStateException e2) {
-        // Bug 39496 (Jrockit related) Give up. The following
+        // Bug 39496 (JRockit related) Give up. The following
         // error is not entirely sane but gives the correct general picture.
         return new CacheClosedException(reason, GemFireCacheImpl.this.disconnectCause);
       }
@@ -1603,7 +1538,7 @@ public class GemFireCacheImpl
 
   @Override
   public CancelCriterion getCancelCriterion() {
-    return stopper;
+    return this.stopper;
   }
 
   /** return true if the cache was closed due to being shunned by other members */
@@ -1676,8 +1611,8 @@ public class GemFireCacheImpl
   public static void emergencyClose() {
     final boolean DEBUG = SystemFailure.TRACE_CLOSE;
 
-    GemFireCacheImpl inst = GemFireCacheImpl.instance;
-    if (inst == null) {
+    GemFireCacheImpl cache = GemFireCacheImpl.instance;
+    if (cache == null) {
       if (DEBUG) {
         System.err.println("GemFireCache#emergencyClose: no instance");
       }
@@ -1687,10 +1622,9 @@ public class GemFireCacheImpl
     GemFireCacheImpl.instance = null;
     GemFireCacheImpl.pdxInstance = null;
     // leave the PdxSerializer set if we have one to prevent 43412
-    // TypeRegistry.setPdxSerializer(null);
 
     // Shut down messaging first
-    InternalDistributedSystem ids = inst.system;
+    InternalDistributedSystem ids = cache.system;
     if (ids != null) {
       if (DEBUG) {
         System.err.println("DEBUG: emergencyClose InternalDistributedSystem");
@@ -1698,20 +1632,18 @@ public class GemFireCacheImpl
       ids.emergencyClose();
     }
 
-    inst.disconnectCause = SystemFailure.getFailure();
-    inst.isClosing = true;
+    cache.disconnectCause = SystemFailure.getFailure();
+    cache.isClosing = true;
 
     // Clear cache servers
     if (DEBUG) {
       System.err.println("DEBUG: Close cache servers");
     }
     {
-      Iterator allCacheServersItr = inst.allCacheServers.iterator();
-      while (allCacheServersItr.hasNext()) {
-        CacheServerImpl bs = (CacheServerImpl) allCacheServersItr.next();
-        AcceptorImpl ai = bs.getAcceptor();
-        if (ai != null) {
-          ai.emergencyClose();
+      for (CacheServerImpl cacheServer : cache.allCacheServers) {
+        AcceptorImpl acceptor = cacheServer.getAcceptor();
+        if (acceptor != null) {
+          acceptor.emergencyClose();
         }
       }
     }
@@ -1725,16 +1657,13 @@ public class GemFireCacheImpl
       System.err.println("DEBUG: closing gateway hubs");
     }
 
-    // These are synchronized sets -- avoid potential deadlocks
-    // instance.pathToRegion.clear(); // garbage collection
-    // instance.gatewayHubs.clear();
-
     // rootRegions is intentionally *not* synchronized. The
     // implementation of clear() does not currently allocate objects.
-    inst.rootRegions.clear();
+    cache.rootRegions.clear();
+
     // partitionedRegions is intentionally *not* synchronized, The
     // implementation of clear() does not currently allocate objects.
-    inst.partitionedRegions.clear();
+    cache.partitionedRegions.clear();
     if (DEBUG) {
       System.err.println("DEBUG: done with cache emergency close");
     }
@@ -1742,7 +1671,7 @@ public class GemFireCacheImpl
 
   @Override
   public boolean isCacheAtShutdownAll() {
-    return isShutDownAll.get();
+    return this.isShutDownAll.get();
   }
 
   /**
@@ -1752,7 +1681,7 @@ public class GemFireCacheImpl
   private static final int shutdownAllPoolSize =
       Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "SHUTDOWN_ALL_POOL_SIZE", -1);
 
-  void shutdownSubTreeGracefully(Map<String, PartitionedRegion> prSubMap) {
+  private void shutdownSubTreeGracefully(Map<String, PartitionedRegion> prSubMap) {
     for (final PartitionedRegion pr : prSubMap.values()) {
       shutDownOnePRGracefully(pr);
     }
@@ -1782,27 +1711,23 @@ public class GemFireCacheImpl
         boolean testIGE = Boolean.getBoolean("TestInternalGemFireError");
 
         if (testIGE) {
-          InternalGemFireError assErr = new InternalGemFireError(
+          throw new InternalGemFireError(
               LocalizedStrings.GemFireCache_UNEXPECTED_EXCEPTION.toLocalizedString());
-          throw assErr;
         }
 
-        // bug 44031 requires multithread shutdownall should be grouped
+        // bug 44031 requires multithread shutDownAll should be grouped
         // by root region. However, shutDownAllDuringRecovery.conf test revealed that
         // we have to close colocated child regions first.
         // Now check all the PR, if anyone has colocate-with attribute, sort all the
         // PRs by colocation relationship and close them sequentially, otherwise still
         // group them by root region.
-        TreeMap<String, Map<String, PartitionedRegion>> prTrees = getPRTrees();
+        SortedMap<String, Map<String, PartitionedRegion>> prTrees = getPRTrees();
         if (prTrees.size() > 1 && shutdownAllPoolSize != 1) {
           ExecutorService es = getShutdownAllExecutorService(prTrees.size());
           for (final Map<String, PartitionedRegion> prSubMap : prTrees.values()) {
-            es.execute(new Runnable() {
-              @Override
-              public void run() {
-                ConnectionTable.threadWantsSharedResources();
-                shutdownSubTreeGracefully(prSubMap);
-              }
+            es.execute(() -> {
+              ConnectionTable.threadWantsSharedResources();
+              shutdownSubTreeGracefully(prSubMap);
             });
           } // for each root
           es.shutdown();
@@ -1827,76 +1752,83 @@ public class GemFireCacheImpl
   }
 
   private ExecutorService getShutdownAllExecutorService(int size) {
-    final ThreadGroup thrGrp = LoggingThreadGroup.createThreadGroup("ShutdownAllGroup", logger);
-    ThreadFactory thrFactory = new ThreadFactory() {
+    final ThreadGroup threadGroup =
+        LoggingThreadGroup.createThreadGroup("ShutdownAllGroup", logger);
+    ThreadFactory threadFactory = new ThreadFactory() {
       private final AtomicInteger threadCount = new AtomicInteger(1);
 
       @Override
-      public Thread newThread(Runnable r) {
-        Thread t = new Thread(thrGrp, r, "ShutdownAll-" + threadCount.getAndIncrement());
-        t.setDaemon(true);
-        return t;
+      public Thread newThread(Runnable runnable) {
+        Thread thread =
+            new Thread(threadGroup, runnable, "ShutdownAll-" + this.threadCount.getAndIncrement());
+        thread.setDaemon(true);
+        return thread;
       }
     };
-    ExecutorService es = Executors
-        .newFixedThreadPool(shutdownAllPoolSize == -1 ? size : shutdownAllPoolSize, thrFactory);
-    return es;
+    return Executors.newFixedThreadPool(shutdownAllPoolSize == -1 ? size : shutdownAllPoolSize,
+        threadFactory);
   }
 
-  private void shutDownOnePRGracefully(PartitionedRegion pr) {
+  private void shutDownOnePRGracefully(PartitionedRegion partitionedRegion) {
     boolean acquiredLock = false;
     try {
-      pr.acquireDestroyLock();
+      partitionedRegion.acquireDestroyLock();
       acquiredLock = true;
 
-      synchronized (pr.getRedundancyProvider()) {
-        if (pr.isDataStore() && pr.getDataStore() != null
-            && pr.getDataPolicy() == DataPolicy.PERSISTENT_PARTITION) {
-          int numBuckets = pr.getTotalNumberOfBuckets();
-          Map<InternalDistributedMember, PersistentMemberID> bucketMaps[] = new Map[numBuckets];
-          PartitionedRegionDataStore prds = pr.getDataStore();
+      synchronized (partitionedRegion.getRedundancyProvider()) {
+        if (partitionedRegion.isDataStore() && partitionedRegion.getDataStore() != null
+            && partitionedRegion.getDataPolicy() == DataPolicy.PERSISTENT_PARTITION) {
+          int numBuckets = partitionedRegion.getTotalNumberOfBuckets();
+          Map<InternalDistributedMember, PersistentMemberID>[] bucketMaps = new Map[numBuckets];
+          PartitionedRegionDataStore dataStore = partitionedRegion.getDataStore();
 
           // lock all the primary buckets
-          Set<Entry<Integer, BucketRegion>> bucketEntries = prds.getAllLocalBuckets();
+          Set<Entry<Integer, BucketRegion>> bucketEntries = dataStore.getAllLocalBuckets();
           for (Map.Entry e : bucketEntries) {
-            BucketRegion br = (BucketRegion) e.getValue();
-            if (br == null || br.isDestroyed) {
+            BucketRegion bucket = (BucketRegion) e.getValue();
+            if (bucket == null || bucket.isDestroyed) {
               // bucket region could be destroyed in race condition
               continue;
             }
-            br.getBucketAdvisor().tryLockIfPrimary();
+            bucket.getBucketAdvisor().tryLockIfPrimary();
 
-            // get map <InternalDistriutedMemeber, persistentID> for this bucket's
+            // get map <InternalDistributedMember, persistentID> for this bucket's
             // remote members
-            bucketMaps[br.getId()] = br.getBucketAdvisor().adviseInitializedPersistentMembers();
+            bucketMaps[bucket.getId()] =
+                bucket.getBucketAdvisor().adviseInitializedPersistentMembers();
             if (logger.isDebugEnabled()) {
               logger.debug("shutDownAll: PR {}: initialized persistent members for {}:{}",
-                  pr.getName(), br.getId(), bucketMaps[br.getId()]);
+                  partitionedRegion.getName(), bucket.getId(), bucketMaps[bucket.getId()]);
             }
           }
           if (logger.isDebugEnabled()) {
-            logger.debug("shutDownAll: All buckets for PR {} are locked.", pr.getName());
+            logger.debug("shutDownAll: All buckets for PR {} are locked.",
+                partitionedRegion.getName());
           }
 
           // send lock profile update to other members
-          pr.setShutDownAllStatus(PartitionedRegion.PRIMARY_BUCKETS_LOCKED);
-          new UpdateAttributesProcessor(pr).distribute(false);
-          pr.getRegionAdvisor().waitForProfileStatus(PartitionedRegion.PRIMARY_BUCKETS_LOCKED);
+          partitionedRegion.setShutDownAllStatus(PartitionedRegion.PRIMARY_BUCKETS_LOCKED);
+          new UpdateAttributesProcessor(partitionedRegion).distribute(false);
+          partitionedRegion.getRegionAdvisor()
+              .waitForProfileStatus(PartitionedRegion.PRIMARY_BUCKETS_LOCKED);
           if (logger.isDebugEnabled()) {
-            logger.debug("shutDownAll: PR {}: all bucketlock profiles received.", pr.getName());
+            logger.debug("shutDownAll: PR {}: all bucketLock profiles received.",
+                partitionedRegion.getName());
           }
 
           // if async write, do flush
-          if (!pr.getAttributes().isDiskSynchronous()) {
-            // several PRs might share the same diskstore, we will only flush once
+          if (!partitionedRegion.getAttributes().isDiskSynchronous()) {
+            // several PRs might share the same diskStore, we will only flush once
             // even flush is called several times.
-            pr.getDiskStore().forceFlush();
+            partitionedRegion.getDiskStore().forceFlush();
             // send flush profile update to other members
-            pr.setShutDownAllStatus(PartitionedRegion.DISK_STORE_FLUSHED);
-            new UpdateAttributesProcessor(pr).distribute(false);
-            pr.getRegionAdvisor().waitForProfileStatus(PartitionedRegion.DISK_STORE_FLUSHED);
+            partitionedRegion.setShutDownAllStatus(PartitionedRegion.DISK_STORE_FLUSHED);
+            new UpdateAttributesProcessor(partitionedRegion).distribute(false);
+            partitionedRegion.getRegionAdvisor()
+                .waitForProfileStatus(PartitionedRegion.DISK_STORE_FLUSHED);
             if (logger.isDebugEnabled()) {
-              logger.debug("shutDownAll: PR {}: all flush profiles received.", pr.getName());
+              logger.debug("shutDownAll: PR {}: all flush profiles received.",
+                  partitionedRegion.getName());
             }
           } // async write
 
@@ -1904,41 +1836,43 @@ public class GemFireCacheImpl
           // iterate through all the bucketMaps and exclude the items whose
           // idm is no longer online
           Set<InternalDistributedMember> membersToPersistOfflineEqual =
-              pr.getRegionAdvisor().adviseDataStore();
+              partitionedRegion.getRegionAdvisor().adviseDataStore();
           for (Map.Entry e : bucketEntries) {
-            BucketRegion br = (BucketRegion) e.getValue();
-            if (br == null || br.isDestroyed) {
+            BucketRegion bucket = (BucketRegion) e.getValue();
+            if (bucket == null || bucket.isDestroyed) {
               // bucket region could be destroyed in race condition
               continue;
             }
             Map<InternalDistributedMember, PersistentMemberID> persistMap =
-                getSubMapForLiveMembers(pr, membersToPersistOfflineEqual, bucketMaps[br.getId()]);
+                getSubMapForLiveMembers(membersToPersistOfflineEqual, bucketMaps[bucket.getId()]);
             if (persistMap != null) {
-              br.getPersistenceAdvisor().persistMembersOfflineAndEqual(persistMap);
+              bucket.getPersistenceAdvisor().persistMembersOfflineAndEqual(persistMap);
               if (logger.isDebugEnabled()) {
-                logger.debug("shutDownAll: PR {}: pesisting bucket {}:{}", pr.getName(), br.getId(),
-                    persistMap);
+                logger.debug("shutDownAll: PR {}: persisting bucket {}:{}",
+                    partitionedRegion.getName(), bucket.getId(), persistMap);
               }
             }
           }
 
-          // send persited profile update to other members, let all members to persist
+          // send persisted profile update to other members, let all members to persist
           // before close the region
-          pr.setShutDownAllStatus(PartitionedRegion.OFFLINE_EQUAL_PERSISTED);
-          new UpdateAttributesProcessor(pr).distribute(false);
-          pr.getRegionAdvisor().waitForProfileStatus(PartitionedRegion.OFFLINE_EQUAL_PERSISTED);
+          partitionedRegion.setShutDownAllStatus(PartitionedRegion.OFFLINE_EQUAL_PERSISTED);
+          new UpdateAttributesProcessor(partitionedRegion).distribute(false);
+          partitionedRegion.getRegionAdvisor()
+              .waitForProfileStatus(PartitionedRegion.OFFLINE_EQUAL_PERSISTED);
           if (logger.isDebugEnabled()) {
-            logger.debug("shutDownAll: PR {}: all offline_equal profiles received.", pr.getName());
+            logger.debug("shutDownAll: PR {}: all offline_equal profiles received.",
+                partitionedRegion.getName());
           }
-        } // datastore
+        } // dataStore
 
-        // after done all steps for buckets, close pr
+        // after done all steps for buckets, close partitionedRegion
         // close accessor directly
-        RegionEventImpl event =
-            new RegionEventImpl(pr, Operation.REGION_CLOSE, null, false, getMyId(), true);
+        RegionEventImpl event = new RegionEventImpl(partitionedRegion, Operation.REGION_CLOSE, null,
+            false, getMyId(), true);
         try {
           // not to acquire lock
-          pr.basicDestroyRegion(event, false, false, true);
+          partitionedRegion.basicDestroyRegion(event, false, false, true);
         } catch (CacheWriterException e) {
           // not possible with local operation, CacheWriter not called
           throw new Error(
@@ -1952,36 +1886,33 @@ public class GemFireCacheImpl
                   .toLocalizedString(),
               e);
         }
-        // pr.close();
       } // synchronized
     } catch (CacheClosedException cce) {
       logger.debug("Encounter CacheClosedException when shutDownAll is closing PR: {}:{}",
-          pr.getFullPath(), cce.getMessage());
+          partitionedRegion.getFullPath(), cce.getMessage());
     } catch (CancelException ce) {
       logger.debug("Encounter CancelException when shutDownAll is closing PR: {}:{}",
-          pr.getFullPath(), ce.getMessage());
+          partitionedRegion.getFullPath(), ce.getMessage());
     } catch (RegionDestroyedException rde) {
       logger.debug("Encounter CacheDestroyedException when shutDownAll is closing PR: {}:{}",
-          pr.getFullPath(), rde.getMessage());
+          partitionedRegion.getFullPath(), rde.getMessage());
     } finally {
       if (acquiredLock) {
-        pr.releaseDestroyLock();
+        partitionedRegion.releaseDestroyLock();
       }
     }
   }
 
-  private Map<InternalDistributedMember, PersistentMemberID> getSubMapForLiveMembers(
-      PartitionedRegion pr, Set<InternalDistributedMember> membersToPersistOfflineEqual,
+  private static Map<InternalDistributedMember, PersistentMemberID> getSubMapForLiveMembers(
+      Set<InternalDistributedMember> membersToPersistOfflineEqual,
       Map<InternalDistributedMember, PersistentMemberID> bucketMap) {
     if (bucketMap == null) {
       return null;
     }
-    Map<InternalDistributedMember, PersistentMemberID> persistMap = new HashMap();
-    Iterator itor = membersToPersistOfflineEqual.iterator();
-    while (itor.hasNext()) {
-      InternalDistributedMember idm = (InternalDistributedMember) itor.next();
-      if (bucketMap.containsKey(idm)) {
-        persistMap.put(idm, bucketMap.get(idm));
+    Map<InternalDistributedMember, PersistentMemberID> persistMap = new HashMap<>();
+    for (InternalDistributedMember member : membersToPersistOfflineEqual) {
+      if (bucketMap.containsKey(member)) {
+        persistMap.put(member, bucketMap.get(member));
       }
     }
     return persistMap;
@@ -1992,13 +1923,13 @@ public class GemFireCacheImpl
     close(false);
   }
 
-  public void close(String reason, boolean keepalive, boolean keepDS) {
-    close(reason, null, keepalive, keepDS);
+  public void close(String reason, boolean keepAlive, boolean keepDS) {
+    close(reason, null, keepAlive, keepDS);
   }
 
   @Override
-  public void close(boolean keepalive) {
-    close("Normal disconnect", null, keepalive, false);
+  public void close(boolean keepAlive) {
+    close("Normal disconnect", null, keepAlive, false);
   }
 
   public void close(String reason, Throwable optionalCause) {
@@ -2086,7 +2017,7 @@ public class GemFireCacheImpl
     }
   }
 
-  public HeapEvictor getHeapEvictor() {
+  public final HeapEvictor getHeapEvictor() {
     synchronized (this.heapEvictorLock) {
       stopper.checkCancelInProgress(null);
       if (this.heapEvictor == null) {
@@ -2096,9 +2027,9 @@ public class GemFireCacheImpl
     }
   }
 
-  public OffHeapEvictor getOffHeapEvictor() {
+  public final OffHeapEvictor getOffHeapEvictor() {
     synchronized (this.offHeapEvictorLock) {
-      stopper.checkCancelInProgress(null);
+      this.stopper.checkCancelInProgress(null);
       if (this.offHeapEvictor == null) {
         this.offHeapEvictor = new OffHeapEvictor(this);
       }
@@ -2108,37 +2039,26 @@ public class GemFireCacheImpl
 
   @Override
   public PersistentMemberManager getPersistentMemberManager() {
-    return persistentMemberManager;
+    return this.persistentMemberManager;
   }
 
   @Override
   public ClientMetadataService getClientMetadataService() {
     synchronized (this.clientMetaDatServiceLock) {
-      stopper.checkCancelInProgress(null);
-      if (this.clientMetadatService == null) {
-        this.clientMetadatService = new ClientMetadataService(this);
+      this.stopper.checkCancelInProgress(null);
+      if (this.clientMetadataService == null) {
+        this.clientMetadataService = new ClientMetadataService(this);
       }
-      return this.clientMetadatService;
+      return this.clientMetadataService;
     }
   }
 
   private final boolean DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE = Boolean
       .getBoolean(DistributionConfig.GEMFIRE_PREFIX + "DISABLE_DISCONNECT_DS_ON_CACHE_CLOSE");
 
-  /**
-   * close the cache
-   *
-   * @param reason the reason the cache is being closed
-   * @param systemFailureCause whether this member was ejected from the distributed system
-   * @param keepalive whoever added this should javadoc it
-   */
-  public void close(String reason, Throwable systemFailureCause, boolean keepalive) {
-    close(reason, systemFailureCause, keepalive, false);
-  }
-
-  public void close(String reason, Throwable systemFailureCause, boolean keepalive,
+  public void close(String reason, Throwable systemFailureCause, boolean keepAlive,
       boolean keepDS) {
-    securityService.close();
+    this.securityService.close();
 
     if (isClosed()) {
       return;
@@ -2146,7 +2066,7 @@ public class GemFireCacheImpl
     final boolean isDebugEnabled = logger.isDebugEnabled();
 
     synchronized (GemFireCacheImpl.class) {
-      // bugfix for bug 36512 "GemFireCache.close is not thread safe"
+      // fix for bug 36512 "GemFireCache.close is not thread safe"
       // ALL CODE FOR CLOSE SHOULD NOW BE UNDER STATIC SYNCHRONIZATION
       // OF synchronized (GemFireCache.class) {
       // static synchronization is necessary due to static resources
@@ -2154,14 +2074,14 @@ public class GemFireCacheImpl
         return;
       }
 
-      /**
+      /*
        * First close the ManagementService as it uses a lot of infra which will be closed by
        * cache.close()
-       **/
+       */
       system.handleResourceEvent(ResourceEvent.CACHE_REMOVE, this);
-      if (this.listener != null) {
-        this.system.removeResourceListener(listener);
-        this.listener = null;
+      if (this.resourceEventsListener != null) {
+        this.system.removeResourceListener(resourceEventsListener);
+        this.resourceEventsListener = null;
       }
 
       if (systemFailureCause != null) {
@@ -2173,7 +2093,7 @@ public class GemFireCacheImpl
         }
       }
 
-      this.keepAlive = keepalive;
+      this.keepAlive = keepAlive;
       isClosing = true;
       logger.info(LocalizedMessage.create(LocalizedStrings.GemFireCache_0_NOW_CLOSING, this));
 
@@ -2191,12 +2111,12 @@ public class GemFireCacheImpl
       TXStateProxy tx = null;
       try {
 
-        if (this.txMgr != null) {
-          tx = this.txMgr.internalSuspend();
+        if (this.transactionManager != null) {
+          tx = this.transactionManager.internalSuspend();
         }
 
         // do this before closing regions
-        resourceManager.close();
+        this.resourceManager.close();
 
         try {
           this.resourceAdvisor.close();
@@ -2209,11 +2129,10 @@ public class GemFireCacheImpl
           // ignore
         }
 
-        GatewaySenderAdvisor advisor = null;
         for (GatewaySender sender : this.getAllGatewaySenders()) {
           try {
             sender.stop();
-            advisor = ((AbstractGatewaySender) sender).getSenderAdvisor();
+            GatewaySenderAdvisor advisor = ((AbstractGatewaySender) sender).getSenderAdvisor();
             if (advisor != null) {
               if (isDebugEnabled) {
                 logger.debug("Stopping the GatewaySender advisor");
@@ -2265,9 +2184,9 @@ public class GemFireCacheImpl
             GemFireCacheImpl.pdxInstance = null;
           }
 
-          List rootRegionValues = null;
+          List<LocalRegion> rootRegionValues;
           synchronized (this.rootRegions) {
-            rootRegionValues = new ArrayList(this.rootRegions.values());
+            rootRegionValues = new ArrayList<>(this.rootRegions.values());
           }
           {
             final Operation op;
@@ -2281,8 +2200,7 @@ public class GemFireCacheImpl
 
             LocalRegion prRoot = null;
 
-            for (Iterator itr = rootRegionValues.iterator(); itr.hasNext();) {
-              LocalRegion lr = (LocalRegion) itr.next();
+            for (LocalRegion lr : rootRegionValues) {
               if (isDebugEnabled) {
                 logger.debug("{}: processing region {}", this, lr.getFullPath());
               }
@@ -2338,7 +2256,7 @@ public class GemFireCacheImpl
                 LocalizedStrings.GemFireCache_FAILED_TO_GET_THE_CQSERVICE_TO_CLOSE_DURING_CACHE_CLOSE_1));
           }
 
-          PoolManager.close(keepalive);
+          PoolManager.close(keepAlive);
 
           if (isDebugEnabled) {
             logger.debug("{}: notifying admins of close...", this);
@@ -2360,7 +2278,7 @@ public class GemFireCacheImpl
           DM dm = null;
           try {
             dm = system.getDistributionManager();
-            dm.removeMembershipListener(this.txMgr);
+            dm.removeMembershipListener(this.transactionManager);
           } catch (CancelException e) {
             // dm = null;
           }
@@ -2390,7 +2308,7 @@ public class GemFireCacheImpl
           // NO MORE Distributed Messaging AFTER THIS POINT!!!!
 
           {
-            ClientMetadataService cms = this.clientMetadatService;
+            ClientMetadataService cms = this.clientMetadataService;
             if (cms != null) {
               cms.close();
             }
@@ -2403,20 +2321,6 @@ public class GemFireCacheImpl
           // make sure the disk stores get closed
           closeDiskStores();
           // NO DISTRIBUTED MESSAGING CAN BE DONE HERE!
-
-          // okay, we're taking too long to do this stuff, so let's
-          // be mean to other processes and skip the rest of the messaging
-          // phase
-          // [bruce] the following code is unnecessary since someone put the
-          // same actions in a finally block
-          // if (!this.closed) {
-          // this.closed = true;
-          // this.txMgr.close();
-          // if (GemFireCache.instance == this) {
-          // GemFireCache.instance = null;
-          // }
-          // ((DynamicRegionFactoryImpl)DynamicRegionFactory.get()).close();
-          // }
         }
 
         // Close the CqService Handle.
@@ -2448,12 +2352,12 @@ public class GemFireCacheImpl
 
       } finally {
         // NO DISTRIBUTED MESSAGING CAN BE DONE HERE!
-        if (this.txMgr != null) {
-          this.txMgr.close();
+        if (this.transactionManager != null) {
+          this.transactionManager.close();
         }
         ((DynamicRegionFactoryImpl) DynamicRegionFactory.get()).close();
-        if (this.txMgr != null) {
-          this.txMgr.internalResume(tx);
+        if (this.transactionManager != null) {
+          this.transactionManager.internalResume(tx);
         }
         TXCommitMessage.getTracker().clearForCacheClose();
       }
@@ -2470,8 +2374,7 @@ public class GemFireCacheImpl
       // do this late to prevent 43412
       TypeRegistry.setPdxSerializer(null);
 
-      for (Iterator iter = cacheLifecycleListeners.iterator(); iter.hasNext();) {
-        CacheLifecycleListener listener = (CacheLifecycleListener) iter.next();
+      for (CacheLifecycleListener listener : cacheLifecycleListeners) {
         listener.cacheClosed(this);
       }
       // Fix for #49856
@@ -2482,13 +2385,11 @@ public class GemFireCacheImpl
 
   }
 
-  // see Cache.isReconnecting()
   @Override
   public boolean isReconnecting() {
     return this.system.isReconnecting();
   }
 
-  // see Cache.waitUntilReconnected(long, TimeUnit)
   @Override
   public boolean waitUntilReconnected(long time, TimeUnit units) throws InterruptedException {
     boolean systemReconnected = this.system.waitUntilReconnected(time, units);
@@ -2496,10 +2397,7 @@ public class GemFireCacheImpl
       return false;
     }
     GemFireCacheImpl cache = getInstance();
-    if (cache == null || !cache.isInitialized()) {
-      return false;
-    }
-    return true;
+    return cache != null && cache.isInitialized();
   }
 
   @Override
@@ -2509,14 +2407,14 @@ public class GemFireCacheImpl
 
   @Override
   public Cache getReconnectedCache() {
-    GemFireCacheImpl c = GemFireCacheImpl.getInstance();
-    if (c == null) {
+    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+    if (cache == null) {
       return null;
     }
-    if (c == this || !c.isInitialized()) {
-      c = null;
+    if (cache == this || !cache.isInitialized()) {
+      cache = null;
     }
-    return c;
+    return cache;
   }
 
   private void stopMemcachedServer() {
@@ -2544,16 +2442,16 @@ public class GemFireCacheImpl
 
   private void prepareDiskStoresForClose() {
     String pdxDSName = TypeRegistry.getPdxDiskStoreName(this);
-    DiskStoreImpl pdxdsi = null;
+    DiskStoreImpl pdxDiskStore = null;
     for (DiskStoreImpl dsi : this.diskStores.values()) {
       if (dsi.getName().equals(pdxDSName)) {
-        pdxdsi = dsi;
+        pdxDiskStore = dsi;
       } else {
         dsi.prepareForClose();
       }
     }
-    if (pdxdsi != null) {
-      pdxdsi.prepareForClose();
+    if (pdxDiskStore != null) {
+      pdxDiskStore.prepareForClose();
     }
   }
 
@@ -2561,48 +2459,33 @@ public class GemFireCacheImpl
    * Used to guard access to compactorPool and set to true when cache is shutdown.
    */
   private final AtomicBoolean diskStoreTaskSync = new AtomicBoolean(false);
+
   /**
-   * Lazily initialized.
+   * Lazily initialized. TODO: this is always null
    */
   private ThreadPoolExecutor diskStoreTaskPool = null;
 
-  private void createDiskStoreTaskPool() {
-    int MAXT = DiskStoreImpl.MAX_CONCURRENT_COMPACTIONS;
-    final ThreadGroup compactThreadGroup =
-        LoggingThreadGroup.createThreadGroup("Oplog Compactor Thread Group", logger);
-    /*
-     * final ThreadFactory compactThreadFactory = new ThreadFactory() { public Thread
-     * newThread(Runnable command) { Thread thread = new Thread(compactThreadGroup, command,
-     * "Idle OplogCompactor"); thread.setDaemon(true); return thread; } };
-     */
-
-    final ThreadFactory compactThreadFactory =
-        GemfireCacheHelper.CreateThreadFactory(compactThreadGroup, "Idle OplogCompactor");
-    this.diskStoreTaskPool = new ThreadPoolExecutor(MAXT, MAXT, 1, TimeUnit.SECONDS,
-        new LinkedBlockingQueue(), compactThreadFactory);
-  }
+  private final ConcurrentMap<String, DiskStoreImpl> diskStores = new ConcurrentHashMap<>();
 
-  private final ConcurrentMap<String, DiskStoreImpl> diskStores =
-      new ConcurrentHashMap<String, DiskStoreImpl>();
   private final ConcurrentMap<String, DiskStoreImpl> regionOwnedDiskStores =
-      new ConcurrentHashMap<String, DiskStoreImpl>();
+      new ConcurrentHashMap<>();
 
-  public void addDiskStore(DiskStoreImpl dsi) {
+  void addDiskStore(DiskStoreImpl dsi) {
     this.diskStores.put(dsi.getName(), dsi);
     if (!dsi.isOffline()) {
       getDiskStoreMonitor().addDiskStore(dsi);
     }
   }
 
-  public void removeDiskStore(DiskStoreImpl dsi) {
+  void removeDiskStore(DiskStoreImpl dsi) {
     this.diskStores.remove(dsi.getName());
     this.regionOwnedDiskStores.remove(dsi.getName());
-    /** Added for M&M **/
+    // Added for M&M
     if (!dsi.getOwnedByRegion())
       system.handleResourceEvent(ResourceEvent.DISKSTORE_REMOVE, dsi);
   }
 
-  public void addRegionOwnedDiskStore(DiskStoreImpl dsi) {
+  void addRegionOwnedDiskStore(DiskStoreImpl dsi) {
     this.regionOwnedDiskStores.put(dsi.getName(), dsi);
     if (!dsi.isOffline()) {
       getDiskStoreMonitor().addDiskStore(dsi);
@@ -2618,7 +2501,7 @@ public class GemFireCacheImpl
           logger.debug("closing {}", dsi);
         }
         dsi.close();
-        /** Added for M&M **/
+        // Added for M&M
         system.handleResourceEvent(ResourceEvent.DISKSTORE_REMOVE, dsi);
       } catch (Exception e) {
         logger.fatal(
@@ -2635,13 +2518,6 @@ public class GemFireCacheImpl
     DEFAULT_DS_NAME = dsName;
   }
 

<TRUNCATED>

[08/13] geode git commit: GEODE-2097: fix flaky test

Posted by kl...@apache.org.
GEODE-2097: fix flaky test


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

Branch: refs/heads/feature/GEODE-2632-6-1
Commit: 6eb9ff36dddeaaea01f624b453b4fa6d308b93f4
Parents: 45dc674
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Fri Apr 21 16:36:14 2017 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon Apr 24 08:59:47 2017 -0700

----------------------------------------------------------------------
 .../OffHeapLRURecoveryRegressionTest.java       | 27 ++++++++++----------
 1 file changed, 14 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/6eb9ff36/geode-core/src/test/java/org/apache/geode/internal/offheap/OffHeapLRURecoveryRegressionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/offheap/OffHeapLRURecoveryRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/offheap/OffHeapLRURecoveryRegressionTest.java
index 489c62b..bb781e6 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/offheap/OffHeapLRURecoveryRegressionTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/offheap/OffHeapLRURecoveryRegressionTest.java
@@ -19,7 +19,9 @@ import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 
+import org.awaitility.Awaitility;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -49,7 +51,6 @@ public class OffHeapLRURecoveryRegressionTest {
   @Test
   public void recoveringTooMuchDataDoesNotRunOutOfOffHeapMemory() {
     final int ENTRY_COUNT = 40;
-    final int expectedObjectCount;
     GemFireCacheImpl gfc = createCache();
     try {
       Region<Object, Object> r = createRegion(gfc);
@@ -57,29 +58,26 @@ public class OffHeapLRURecoveryRegressionTest {
       for (int i = 0; i < ENTRY_COUNT; i++) {
         r.put(i, v);
       }
-      // expect one more during recovery because of the way the LRU limit is
-      // enforced during recover.
-      expectedObjectCount = MemoryAllocatorImpl.getAllocator().getStats().getObjects() + 1;
     } finally {
-      gfc.close();
+      closeCache(gfc);
     }
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+      return MemoryAllocatorImpl.getAllocator().getStats().getObjects() == 0;
+    });
     System.setProperty("gemfire.disk.recoverValuesSync", "true");
     System.setProperty("gemfire.disk.recoverLruValues", "true");
     try {
       gfc = createCache();
       try {
-        Region<Object, Object> r = createRegion(gfc);
+        createDiskStore(gfc);
         try {
-          assertEquals(ENTRY_COUNT, r.size());
-          assertEquals(expectedObjectCount,
-              MemoryAllocatorImpl.getAllocator().getStats().getObjects());
+          assertEquals(10, MemoryAllocatorImpl.getAllocator().getStats().getObjects());
         } finally {
-          r.destroyRegion();
           DiskStore ds = gfc.findDiskStore(DS_NAME);
           ds.destroy();
         }
       } finally {
-        gfc.close();
+        closeCache(gfc);
       }
     } finally {
       System.clearProperty("gemfire.disk.recoverValuesSync");
@@ -97,9 +95,13 @@ public class OffHeapLRURecoveryRegressionTest {
     return result;
   }
 
-  private Region<Object, Object> createRegion(GemFireCacheImpl gfc) {
+  private void createDiskStore(GemFireCacheImpl gfc) {
     DiskStoreFactory dsf = gfc.createDiskStoreFactory();
     dsf.create(DS_NAME);
+  }
+
+  private Region<Object, Object> createRegion(GemFireCacheImpl gfc) {
+    createDiskStore(gfc);
     RegionFactory<Object, Object> rf =
         gfc.createRegionFactory(RegionShortcut.LOCAL_PERSISTENT_OVERFLOW);
     rf.setOffHeap(true);
@@ -110,6 +112,5 @@ public class OffHeapLRURecoveryRegressionTest {
 
   private void closeCache(GemFireCacheImpl gfc) {
     gfc.close();
-    MemoryAllocatorImpl.freeOffHeapMemory();
   }
 }


[03/13] geode git commit: GEODE-2632: refactor code to use InternalCache instead of GemFireCacheImpl

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceVsdStats.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceVsdStats.java b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceVsdStats.java
index 8435c4c..5dc7bb0 100644
--- a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceVsdStats.java
+++ b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/CqServiceVsdStats.java
@@ -25,9 +25,9 @@ import org.apache.geode.cache.query.CqException;
 import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.internal.DefaultQueryService;
 import org.apache.geode.internal.NanoTimer;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl;
 import org.apache.geode.internal.cache.FilterProfile;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.logging.LogService;
 
 /**
@@ -44,35 +44,34 @@ public class CqServiceVsdStats {
   private static final StatisticsType _type;
 
   /** Name of the created CQs statistic */
-  protected static final String CQS_CREATED = "numCqsCreated";
+  private static final String CQS_CREATED = "numCqsCreated";
 
   /** Name of the active CQs statistic */
-  protected static final String CQS_ACTIVE = "numCqsActive";
+  private static final String CQS_ACTIVE = "numCqsActive";
 
   /** Name of the stopped CQs statistic */
-  protected static final String CQS_STOPPED = "numCqsStopped";
+  private static final String CQS_STOPPED = "numCqsStopped";
 
   /** Name of the closed CQs statistic */
-  protected static final String CQS_CLOSED = "numCqsClosed";
+  private static final String CQS_CLOSED = "numCqsClosed";
 
   /** Name of the client's CQs statistic */
-  protected static final String CQS_ON_CLIENT = "numCqsOnClient";
+  private static final String CQS_ON_CLIENT = "numCqsOnClient";
 
   /** Number of clients with CQs statistic */
-  protected static final String CLIENTS_WITH_CQS = "numClientsWithCqs";
-
+  private static final String CLIENTS_WITH_CQS = "numClientsWithCqs";
 
   /** CQ query execution time. */
-  protected static final String CQ_QUERY_EXECUTION_TIME = "cqQueryExecutionTime";
+  private static final String CQ_QUERY_EXECUTION_TIME = "cqQueryExecutionTime";
 
   /** CQ query execution in progress */
-  protected static final String CQ_QUERY_EXECUTION_IN_PROGRESS = "cqQueryExecutionInProgress";
+  private static final String CQ_QUERY_EXECUTION_IN_PROGRESS = "cqQueryExecutionInProgress";
 
   /** Completed CQ query executions */
-  protected static final String CQ_QUERY_EXECUTIONS_COMPLETED = "cqQueryExecutionsCompleted";
+  private static final String CQ_QUERY_EXECUTIONS_COMPLETED = "cqQueryExecutionsCompleted";
 
   /** Unique CQs, number of different CQ queries */
-  protected static final String UNIQUE_CQ_QUERY = "numUniqueCqQuery";
+  private static final String UNIQUE_CQ_QUERY = "numUniqueCqQuery";
 
   /** Id of the CQs created statistic */
   private static final int _numCqsCreatedId;
@@ -104,7 +103,7 @@ public class CqServiceVsdStats {
   /** Id for unique CQs, difference in CQ queries */
   private static final int _numUniqueCqQuery;
 
-  /**
+  /*
    * Static initializer to create and initialize the <code>StatisticsType</code>
    */
   static {
@@ -140,7 +139,6 @@ public class CqServiceVsdStats {
     _cqQueryExecutionsCompletedId = _type.nameToId(CQ_QUERY_EXECUTIONS_COMPLETED);
     _cqQueryExecutionInProgressId = _type.nameToId(CQ_QUERY_EXECUTION_IN_PROGRESS);
     _numUniqueCqQuery = _type.nameToId(UNIQUE_CQ_QUERY);
-
   }
 
   /** The <code>Statistics</code> instance to which most behavior is delegated */
@@ -152,12 +150,10 @@ public class CqServiceVsdStats {
    * @param factory The <code>StatisticsFactory</code> which creates the <code>Statistics</code>
    *        instance
    */
-  public CqServiceVsdStats(StatisticsFactory factory) {
+  CqServiceVsdStats(StatisticsFactory factory) {
     this._stats = factory.createAtomicStatistics(_type, "CqServiceStats");
   }
 
-  // /////////////////// Instance Methods /////////////////////
-
   /**
    * Closes the <code>HARegionQueueStats</code>.
    */
@@ -170,14 +166,14 @@ public class CqServiceVsdStats {
    * 
    * @return the current value of the "numCqsCreated" stat
    */
-  public long getNumCqsCreated() {
+  long getNumCqsCreated() {
     return this._stats.getLong(_numCqsCreatedId);
   }
 
   /**
    * Increments the "numCqsCreated" stat by 1.
    */
-  public void incCqsCreated() {
+  void incCqsCreated() {
     this._stats.incLong(_numCqsCreatedId, 1);
   }
 
@@ -186,21 +182,21 @@ public class CqServiceVsdStats {
    * 
    * @return the current value of the "numCqsActive" stat
    */
-  public long getNumCqsActive() {
+  long getNumCqsActive() {
     return this._stats.getLong(_numCqsActiveId);
   }
 
   /**
    * Increments the "numCqsActive" stat by 1.
    */
-  public void incCqsActive() {
+  void incCqsActive() {
     this._stats.incLong(_numCqsActiveId, 1);
   }
 
   /**
    * Decrements the "numCqsActive" stat by 1.
    */
-  public void decCqsActive() {
+  void decCqsActive() {
     this._stats.incLong(_numCqsActiveId, -1);
   }
 
@@ -209,21 +205,21 @@ public class CqServiceVsdStats {
    * 
    * @return the current value of the "numCqsStopped" stat
    */
-  public long getNumCqsStopped() {
+  long getNumCqsStopped() {
     return this._stats.getLong(_numCqsStoppedId);
   }
 
   /**
    * Increments the "numCqsStopped" stat by 1.
    */
-  public void incCqsStopped() {
+  void incCqsStopped() {
     this._stats.incLong(_numCqsStoppedId, 1);
   }
 
   /**
    * Decrements the "numCqsStopped" stat by 1.
    */
-  public void decCqsStopped() {
+  void decCqsStopped() {
     this._stats.incLong(_numCqsStoppedId, -1);
   }
 
@@ -232,14 +228,14 @@ public class CqServiceVsdStats {
    * 
    * @return the current value of the "numCqsClosed" stat
    */
-  public long getNumCqsClosed() {
+  long getNumCqsClosed() {
     return this._stats.getLong(_numCqsClosedId);
   }
 
   /**
    * Increments the "numCqsClosed" stat by 1.
    */
-  public void incCqsClosed() {
+  void incCqsClosed() {
     this._stats.incLong(_numCqsClosedId, 1);
   }
 
@@ -248,21 +244,21 @@ public class CqServiceVsdStats {
    * 
    * @return the current value of the "numCqsOnClient" stat
    */
-  public long getNumCqsOnClient() {
+  long getNumCqsOnClient() {
     return this._stats.getLong(_numCqsOnClientId);
   }
 
   /**
    * Increments the "numCqsOnClient" stat by 1.
    */
-  public void incCqsOnClient() {
+  void incCqsOnClient() {
     this._stats.incLong(_numCqsOnClientId, 1);
   }
 
   /**
    * Decrements the "numCqsOnClient" stat by 1.
    */
-  public void decCqsOnClient() {
+  void decCqsOnClient() {
     this._stats.incLong(_numCqsOnClientId, -1);
   }
 
@@ -278,21 +274,21 @@ public class CqServiceVsdStats {
   /**
    * Increments the "numClientsWithCqs" stat by 1.
    */
-  public void incClientsWithCqs() {
+  void incClientsWithCqs() {
     this._stats.incLong(_numClientsWithCqsId, 1);
   }
 
   /**
    * Decrements the "numCqsOnClient" stat by 1.
    */
-  public void decClientsWithCqs() {
+  void decClientsWithCqs() {
     this._stats.incLong(_numClientsWithCqsId, -1);
   }
 
   /**
    * Start the CQ Query Execution time.
    */
-  public long startCqQueryExecution() {
+  long startCqQueryExecution() {
     this._stats.incInt(_cqQueryExecutionInProgressId, 1);
     return NanoTimer.getTime();
   }
@@ -302,7 +298,7 @@ public class CqServiceVsdStats {
    * 
    * @param start long time value.
    */
-  public void endCqQueryExecution(long start) {
+  void endCqQueryExecution(long start) {
     long ts = NanoTimer.getTime();
     this._stats.incLong(_cqQueryExecutionTimeId, ts - start);
     this._stats.incInt(_cqQueryExecutionInProgressId, -1);
@@ -321,14 +317,14 @@ public class CqServiceVsdStats {
   /**
    * Increments number of Unique queries.
    */
-  public void incUniqueCqQuery() {
+  void incUniqueCqQuery() {
     this._stats.incInt(_numUniqueCqQuery, 1);
   }
 
   /**
    * Decrements number of unique Queries.
    */
-  public void decUniqueCqQuery() {
+  void decUniqueCqQuery() {
     this._stats.incInt(_numUniqueCqQuery, -1);
   }
 
@@ -338,11 +334,8 @@ public class CqServiceVsdStats {
    * tests.
    * <p>
    * Returns the number of CQs (active + suspended) on the given region.
-   * 
-   * @param regionName
    */
-  public long numCqsOnRegion(String regionName) {
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+  public long numCqsOnRegion(final InternalCache cache, String regionName) {
     if (cache == null) {
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ServerCQImpl.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ServerCQImpl.java b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ServerCQImpl.java
index ec6e984..c484105 100644
--- a/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ServerCQImpl.java
+++ b/geode-cq/src/main/java/org/apache/geode/cache/query/internal/cq/ServerCQImpl.java
@@ -21,24 +21,18 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
-import java.util.concurrent.ConcurrentLinkedQueue;
 
 import org.apache.logging.log4j.Logger;
 
-import org.apache.geode.CancelException;
 import org.apache.geode.DataSerializable;
 import org.apache.geode.DataSerializer;
-import org.apache.geode.SystemFailure;
-import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.EvictionAction;
-import org.apache.geode.cache.client.internal.UserAttributes;
 import org.apache.geode.cache.query.CqAttributes;
 import org.apache.geode.cache.query.CqAttributesMutator;
 import org.apache.geode.cache.query.CqClosedException;
 import org.apache.geode.cache.query.CqException;
 import org.apache.geode.cache.query.CqExistsException;
-import org.apache.geode.cache.query.CqListener;
 import org.apache.geode.cache.query.CqResults;
 import org.apache.geode.cache.query.Query;
 import org.apache.geode.cache.query.QueryException;
@@ -49,6 +43,7 @@ import org.apache.geode.cache.query.internal.CompiledRegion;
 import org.apache.geode.cache.query.internal.CompiledSelect;
 import org.apache.geode.cache.query.internal.CqStateImpl;
 import org.apache.geode.cache.query.internal.DefaultQuery;
+import org.apache.geode.i18n.StringId;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.Token;
@@ -58,7 +53,6 @@ import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
-import org.apache.geode.i18n.StringId;
 
 public class ServerCQImpl extends CqQueryImpl implements DataSerializable, ServerCQ {
   private static final Logger logger = LogService.getLogger();
@@ -84,7 +78,7 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
   public volatile boolean cqResultKeysInitialized = false;
 
   /** Boolean flag to see if the CQ is on Partitioned Region */
-  public volatile boolean isPR = false;
+  volatile boolean isPR = false;
 
   private ClientProxyMembershipID clientProxyId = null;
 
@@ -92,7 +86,6 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
 
   private String serverCqName;
 
-
   /** identifier assigned to this query for FilterRoutingInfos */
   private Long filterID;
 
@@ -106,21 +99,11 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
     // For deserialization
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqQuery2#getFilterID()
-   */
   @Override
   public Long getFilterID() {
     return this.filterID;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqQuery2#setFilterID(java.lang.Long)
-   */
   @Override
   public void setFilterID(Long filterID) {
     this.filterID = filterID;
@@ -142,19 +125,12 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
 
     CacheClientProxy clientProxy = null;
     this.clientProxyId = p_clientProxyId;
-    // servConnection = serverSideConnection;
 
     if (p_ccn != null) {
       this.ccn = p_ccn;
       clientProxy = p_ccn.getClientProxy(p_clientProxyId, true);
     }
 
-    /*
-     * try { initCq(); } catch (CqExistsException cqe) { // Should not happen. throw new
-     * CqException(LocalizedStrings.CqQueryImpl_UNABLE_TO_CREATE_CQ_0_ERROR__1.toLocalizedString(new
-     * Object[] { cqName, cqe.getMessage()})); }
-     */
-
     validateCq();
 
     final boolean isDebugEnabled = logger.isDebugEnabled();
@@ -228,13 +204,11 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
       throw new CqException(errMsg);
     }
 
-    // checkAndSetCqOnRegion();
-
     // Can be null by the time we are here
     if (clientProxy != null) {
       clientProxy.incCqCount();
       if (clientProxy.hasOneCq()) {
-        cqService.stats.incClientsWithCqs();
+        cqService.stats().incClientsWithCqs();
       }
       if (isDebugEnabled) {
         logger.debug("Added CQ to the base region: {} With key as: {}", cqBaseRegion.getFullPath(),
@@ -307,7 +281,6 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
    * query.
    * 
    * @return String modified query.
-   * @throws CqException
    */
   private Query constructServerSideQuery() throws QueryException {
     GemFireCacheImpl cache = (GemFireCacheImpl) cqService.getCache();
@@ -328,7 +301,6 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
    * Returns if the passed key is part of the CQs result set. This method needs to be called once
    * the CQ result key caching is completed (cqResultsCacheInitialized is true).
    * 
-   * @param key
    * @return true if key is in the Results Cache.
    */
   public boolean isPartOfCqResult(Object key) {
@@ -352,27 +324,18 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
     }
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqQuery2#addToCqResultKeys(java.lang.Object)
-   */
   @Override
   public void addToCqResultKeys(Object key) {
     if (!CqServiceProvider.MAINTAIN_KEYS) {
       return;
     }
 
-    // this.logger.fine("Adding key to Results Cache For CQ :" +
-    // this.cqName + " key :" + key);
     if (this.cqResultKeys != null) {
       synchronized (this.cqResultKeys) {
         this.cqResultKeys.put(key, TOKEN);
         if (!this.cqResultKeysInitialized) {
           // This key could be coming after add, destroy.
           // Remove this from destroy queue.
-          // this.logger.fine("Removing key from Destroy Cache For CQ :" +
-          // this.cqName + " key :" + key);
           if (this.destroysWhileCqResultsInProgress != null) {
             this.destroysWhileCqResultsInProgress.remove(key);
           }
@@ -381,21 +344,11 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
     }
   }
 
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqQuery2#removeFromCqResultKeys(java.lang.Object,
-   * boolean)
-   */
   @Override
   public void removeFromCqResultKeys(Object key, boolean isTokenMode) {
     if (!CqServiceProvider.MAINTAIN_KEYS) {
       return;
     }
-    // this.logger.fine("Removing key from Results Cache For CQ :" +
-    // this.cqName + " key :" + key);
     if (this.cqResultKeys != null) {
       synchronized (this.cqResultKeys) {
         if (isTokenMode && this.cqResultKeys.get(key) != Token.DESTROYED) {
@@ -403,8 +356,6 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
         }
         this.cqResultKeys.remove(key);
         if (!this.cqResultKeysInitialized) {
-          // this.logger.fine("Adding key to Destroy Cache For CQ :" +
-          // this.cqName + " key :" + key);
           if (this.destroysWhileCqResultsInProgress != null) {
             this.destroysWhileCqResultsInProgress.add(key);
           }
@@ -415,10 +366,8 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
 
   /**
    * Marks the key as destroyed in the CQ Results key cache.
-   * 
-   * @param key
    */
-  public void markAsDestroyedInCqResultKeys(Object key) {
+  void markAsDestroyedInCqResultKeys(Object key) {
     if (!CqServiceProvider.MAINTAIN_KEYS) {
       return;
     }
@@ -439,12 +388,6 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
     }
   }
 
-
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqQuery2#setCqResultsCacheInitialized()
-   */
   @Override
   public void setCqResultsCacheInitialized() {
     if (CqServiceProvider.MAINTAIN_KEYS) {
@@ -466,13 +409,6 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
     }
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.cache.query.internal.InternalCqQuery2#isOldValueRequiredForQueryProcessing(
-   * java.lang.Object)
-   */
   @Override
   public boolean isOldValueRequiredForQueryProcessing(Object key) {
     if (this.cqResultKeysInitialized && this.isPartOfCqResult(key)) {
@@ -484,18 +420,11 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
   /**
    * Closes the Query. On Client side, sends the cq close request to server. On Server side, takes
    * care of repository cleanup.
-   * 
-   * @throws CqException
    */
   public void close() throws CqClosedException, CqException {
     close(true);
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqQuery2#close(boolean)
-   */
   @Override
   public void close(boolean sendRequestToServer) throws CqClosedException, CqException {
     final boolean isDebugEnabled = logger.isDebugEnabled();
@@ -523,9 +452,9 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
 
       // Stat update.
       if (stateBeforeClosing == CqStateImpl.RUNNING) {
-        cqService.stats.decCqsActive();
+        cqService.stats().decCqsActive();
       } else if (stateBeforeClosing == CqStateImpl.STOPPED) {
-        cqService.stats.decCqsStopped();
+        cqService.stats().decCqsStopped();
       }
 
       // Clean-up the CQ Results Cache.
@@ -537,8 +466,8 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
 
       // Set the state to close, and update stats
       this.cqState.setState(CqStateImpl.CLOSED);
-      cqService.stats.incCqsClosed();
-      cqService.stats.decCqsOnClient();
+      cqService.stats().incCqsClosed();
+      cqService.stats().decCqsOnClient();
       if (this.stats != null)
         this.stats.close();
     }
@@ -564,9 +493,8 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
 
   /**
    * Clears the resource used by CQ.
-   * 
-   * @throws CqException
    */
+  @Override
   protected void cleanup() throws CqException {
     // CqBaseRegion
     try {
@@ -575,7 +503,7 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
         CacheClientProxy clientProxy = ccn.getClientProxy(clientProxyId);
         clientProxy.decCqCount();
         if (clientProxy.hasNoCq()) {
-          cqService.stats.decClientsWithCqs();
+          cqService.stats().decClientsWithCqs();
         }
       }
     } catch (Exception ex) {
@@ -587,16 +515,9 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
   }
 
   /**
-   * @param serverCqName The serverCqName to set.
-   */
-  public void setServerCqName(String serverCqName) {
-
-    this.serverCqName = serverCqName;
-  }
-
-  /**
    * Stop or pause executing the query.
    */
+  @Override
   public void stop() throws CqClosedException, CqException {
     boolean isStopped = false;
     synchronized (this.cqState) {
@@ -613,18 +534,16 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
 
       // Change state and stats on the client side
       this.cqState.setState(CqStateImpl.STOPPED);
-      this.cqService.stats.incCqsStopped();
-      this.cqService.stats.decCqsActive();
+      this.cqService.stats().incCqsStopped();
+      this.cqService.stats().decCqsActive();
       if (logger.isDebugEnabled()) {
         logger.debug("Successfully stopped the CQ. {}", cqName);
       }
     }
   }
 
-  /* DataSerializableFixedID methods ---------------------------------------- */
-
+  @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    // this.cqName = DataSerializer.readString(in);
     synchronized (cqState) {
       this.cqState.setState(DataSerializer.readInteger(in));
     }
@@ -633,23 +552,14 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
     this.filterID = in.readLong();
   }
 
-  /*
-   * public int getDSFID() { return CQ_QUERY; }
-   */
-
+  @Override
   public void toData(DataOutput out) throws IOException {
-    // DataSerializer.writeString(this.cqName, out);
     DataSerializer.writeInteger(this.cqState.getState(), out);
     DataSerializer.writeBoolean(this.isDurable, out);
     DataSerializer.writeString(this.queryString, out);
     out.writeLong(this.filterID);
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.query.internal.InternalCqQuery2#isPR()
-   */
   @Override
   public boolean isPR() {
     return isPR;
@@ -676,5 +586,4 @@ public class ServerCQImpl extends CqQueryImpl implements DataSerializable, Serve
     throw new IllegalStateException("Execute cannot be called on a CQ on the server");
   }
 
-
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ.java b/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ.java
index bcf9806..9bddbc7 100644
--- a/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ.java
+++ b/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ.java
@@ -27,7 +27,6 @@ import org.apache.geode.cache.query.internal.DefaultQueryService;
 import org.apache.geode.cache.query.internal.cq.CqService;
 import org.apache.geode.cache.query.internal.cq.ServerCQ;
 import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.cache.tier.Command;
 import org.apache.geode.internal.cache.tier.MessageType;
@@ -85,8 +84,7 @@ public class ExecuteCQ extends BaseCQCommand {
     ServerCQ cqQuery = null;
 
     try {
-      qService =
-          (DefaultQueryService) ((GemFireCacheImpl) crHelper.getCache()).getLocalQueryService();
+      qService = (DefaultQueryService) crHelper.getCache().getLocalQueryService();
 
       // Authorization check
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ61.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ61.java b/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ61.java
index f333b4b..de61445 100755
--- a/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ61.java
+++ b/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/ExecuteCQ61.java
@@ -28,7 +28,6 @@ import org.apache.geode.cache.query.internal.cq.CqServiceImpl;
 import org.apache.geode.cache.query.internal.cq.CqServiceProvider;
 import org.apache.geode.cache.query.internal.cq.ServerCQImpl;
 import org.apache.geode.distributed.internal.DistributionStats;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.cache.tier.Command;
 import org.apache.geode.internal.cache.tier.MessageType;
@@ -111,8 +110,7 @@ public class ExecuteCQ61 extends BaseCQCommand {
     ServerCQImpl cqQuery = null;
 
     try {
-      qService =
-          (DefaultQueryService) ((GemFireCacheImpl) crHelper.getCache()).getLocalQueryService();
+      qService = (DefaultQueryService) crHelper.getCache().getLocalQueryService();
 
       // Authorization check
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetDurableCQs.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetDurableCQs.java b/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetDurableCQs.java
index eac9ed3..a2d201d 100755
--- a/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetDurableCQs.java
+++ b/geode-cq/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GetDurableCQs.java
@@ -22,7 +22,6 @@ import java.util.List;
 import org.apache.geode.cache.query.CqException;
 import org.apache.geode.cache.query.internal.DefaultQueryService;
 import org.apache.geode.cache.query.internal.cq.CqService;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.tier.CachedRegionHelper;
 import org.apache.geode.internal.cache.tier.Command;
 import org.apache.geode.internal.cache.tier.MessageType;
@@ -64,8 +63,7 @@ public class GetDurableCQs extends BaseCQCommand {
     CqService cqServiceForExec = null;
 
     try {
-      qService =
-          (DefaultQueryService) ((GemFireCacheImpl) crHelper.getCache()).getLocalQueryService();
+      qService = (DefaultQueryService) crHelper.getCache().getLocalQueryService();
 
       this.securityService.authorizeClusterRead();
 

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsDUnitTest.java b/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsDUnitTest.java
index 7ace0e8..f4cd706 100644
--- a/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsDUnitTest.java
+++ b/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsDUnitTest.java
@@ -14,20 +14,15 @@
  */
 package org.apache.geode.cache.query.cq.dunit;
 
-import org.junit.experimental.categories.Category;
-import org.junit.Test;
-
 import static org.junit.Assert.*;
 
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
-
 import java.util.Collection;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.cache.CacheException;
 import org.apache.geode.cache.query.CqException;
-import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.CqServiceStatistics;
 import org.apache.geode.cache.query.CqStatistics;
 import org.apache.geode.cache.query.QueryService;
@@ -40,7 +35,7 @@ import org.apache.geode.cache.query.internal.cq.CqServiceImpl;
 import org.apache.geode.cache.query.internal.cq.CqServiceVsdStats;
 import org.apache.geode.cache.query.internal.cq.InternalCqQuery;
 import org.apache.geode.cache30.CacheSerializableRunnable;
-import org.apache.geode.cache30.CacheTestCase;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.LogWriterUtils;
@@ -48,27 +43,26 @@ import org.apache.geode.test.dunit.NetworkUtils;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.Wait;
+import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
 
 /**
- * This class tests the ContiunousQuery mechanism in GemFire. This includes the test with different
+ * This class tests the ContinuousQuery mechanism in GemFire. This includes the test with different
  * data activities.
- *
  */
 @Category(DistributedTest.class)
 public class CqStatsDUnitTest extends JUnit4CacheTestCase {
 
+  // TODO: delete this use of CqQueryDUnitTest
   private CqQueryDUnitTest cqDUnitTest = new CqQueryDUnitTest();
 
-  public CqStatsDUnitTest() {
-    super();
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     // avoid IllegalStateException from HandShake by connecting all vms to
     // system before creating pool
     getSystem();
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
+      @Override
       public void run() {
         getSystem();
       }
@@ -81,6 +75,7 @@ public class CqStatsDUnitTest extends JUnit4CacheTestCase {
   public void validateCQStats(VM vm, final String cqName, final int creates, final int updates,
       final int deletes, final int totalEvents, final int cqListenerInvocations) {
     vm.invoke(new CacheSerializableRunnable("Validate CQs") {
+      @Override
       public void run2() throws CacheException {
         LogWriterUtils.getLogWriter().info("### Validating CQ Stats. ### " + cqName);
         // Get CQ Service.
@@ -161,6 +156,7 @@ public class CqStatsDUnitTest extends JUnit4CacheTestCase {
       final int stopped, final int closed, final int cqsOnClient, final int cqsOnRegion,
       final int clientsWithCqs) {
     vm.invoke(new CacheSerializableRunnable("Validate CQ Service Stats") {
+      @Override
       public void run2() throws CacheException {
         LogWriterUtils.getLogWriter().info("### Validating CQ Service Stats. ### ");
         // Get CQ Service.
@@ -176,7 +172,7 @@ public class CqStatsDUnitTest extends JUnit4CacheTestCase {
         CqServiceVsdStats cqServiceVsdStats = null;
         try {
           cqServiceVsdStats =
-              ((CqServiceImpl) ((DefaultQueryService) qService).getCqService()).stats;
+              ((CqServiceImpl) ((DefaultQueryService) qService).getCqService()).stats();
         } catch (CqException e) {
           // TODO Auto-generated catch block
           e.printStackTrace();
@@ -185,12 +181,14 @@ public class CqStatsDUnitTest extends JUnit4CacheTestCase {
           fail("Failed to get CQ Service Stats");
         }
 
-        getCache().getLogger().info("#### CQ Service stats: " + " CQs created: "
-            + cqServiceStats.numCqsCreated() + " CQs active: " + cqServiceStats.numCqsActive()
-            + " CQs stopped: " + cqServiceStats.numCqsStopped() + " CQs closed: "
-            + cqServiceStats.numCqsClosed() + " CQs on Client: " + cqServiceStats.numCqsOnClient()
-            + " CQs on region /root/regionA : " + cqServiceVsdStats.numCqsOnRegion("/root/regionA")
-            + " Clients with CQs: " + cqServiceVsdStats.getNumClientsWithCqs());
+        getCache().getLogger()
+            .info("#### CQ Service stats: " + " CQs created: " + cqServiceStats.numCqsCreated()
+                + " CQs active: " + cqServiceStats.numCqsActive() + " CQs stopped: "
+                + cqServiceStats.numCqsStopped() + " CQs closed: " + cqServiceStats.numCqsClosed()
+                + " CQs on Client: " + cqServiceStats.numCqsOnClient()
+                + " CQs on region /root/regionA : "
+                + cqServiceVsdStats.numCqsOnRegion(GemFireCacheImpl.getInstance(), "/root/regionA")
+                + " Clients with CQs: " + cqServiceVsdStats.getNumClientsWithCqs());
 
 
         // Check for created count.
@@ -223,7 +221,7 @@ public class CqStatsDUnitTest extends JUnit4CacheTestCase {
         // Check for CQs on region.
         if (cqsOnRegion != CqQueryDUnitTest.noTest) {
           assertEquals("Number of CQs on region /root/regionA mismatch", cqsOnRegion,
-              cqServiceVsdStats.numCqsOnRegion("/root/regionA"));
+              cqServiceVsdStats.numCqsOnRegion(GemFireCacheImpl.getInstance(), "/root/regionA"));
         }
 
         // Check for clients with CQs count.

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java b/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java
index d6068f1..c03bb8b 100644
--- a/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java
+++ b/geode-cq/src/test/java/org/apache/geode/cache/query/cq/dunit/CqStatsUsingPoolDUnitTest.java
@@ -14,21 +14,16 @@
  */
 package org.apache.geode.cache.query.cq.dunit;
 
-import org.junit.experimental.categories.Category;
-import org.junit.Test;
-
 import static org.junit.Assert.*;
 
-import org.apache.geode.distributed.*;
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
+import java.util.Collection;
+import java.util.Properties;
 
-import java.util.*;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.CacheException;
 import org.apache.geode.cache.query.CqException;
-import org.apache.geode.cache.query.CqQuery;
 import org.apache.geode.cache.query.CqServiceStatistics;
 import org.apache.geode.cache.query.CqStatistics;
 import org.apache.geode.cache.query.QueryService;
@@ -41,7 +36,8 @@ import org.apache.geode.cache.query.internal.cq.CqServiceImpl;
 import org.apache.geode.cache.query.internal.cq.CqServiceVsdStats;
 import org.apache.geode.cache.query.internal.cq.InternalCqQuery;
 import org.apache.geode.cache30.CacheSerializableRunnable;
-import org.apache.geode.cache30.CacheTestCase;
+import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.LogWriterUtils;
@@ -49,21 +45,19 @@ import org.apache.geode.test.dunit.NetworkUtils;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.Wait;
+import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
 
 /**
- * This class tests the ContiunousQuery mechanism in GemFire. This includes the test with different
+ * This class tests the ContinuousQuery mechanism in GemFire. This includes the test with different
  * data activities.
- *
  */
 @Category(DistributedTest.class)
 public class CqStatsUsingPoolDUnitTest extends JUnit4CacheTestCase {
 
+  // TODO: delete this use of CqQueryUsingPoolDUnitTest
   private CqQueryUsingPoolDUnitTest cqDUnitTest = new CqQueryUsingPoolDUnitTest();
 
-  public CqStatsUsingPoolDUnitTest() {
-    super();
-  }
-
   @Override
   public Properties getDistributedSystemProperties() {
     Properties result = super.getDistributedSystemProperties();
@@ -77,6 +71,7 @@ public class CqStatsUsingPoolDUnitTest extends JUnit4CacheTestCase {
     // system before creating pool
     getSystem();
     Invoke.invokeInEveryVM(new SerializableRunnable("getSystem") {
+      @Override
       public void run() {
         getSystem();
       }
@@ -89,6 +84,7 @@ public class CqStatsUsingPoolDUnitTest extends JUnit4CacheTestCase {
   private void validateCQStats(VM vm, final String cqName, final int creates, final int updates,
       final int deletes, final int totalEvents, final int cqListenerInvocations) {
     vm.invoke(new CacheSerializableRunnable("Validate CQs") {
+      @Override
       public void run2() throws CacheException {
         LogWriterUtils.getLogWriter().info("### Validating CQ Stats. ### " + cqName);
         // Get CQ Service.
@@ -169,6 +165,7 @@ public class CqStatsUsingPoolDUnitTest extends JUnit4CacheTestCase {
       final int stopped, final int closed, final int cqsOnClient, final int cqsOnRegion,
       final int clientsWithCqs) {
     vm.invoke(new CacheSerializableRunnable("Validate CQ Service Stats") {
+      @Override
       public void run2() throws CacheException {
         LogWriterUtils.getLogWriter().info("### Validating CQ Service Stats. ### ");
         // Get CQ Service.
@@ -184,7 +181,7 @@ public class CqStatsUsingPoolDUnitTest extends JUnit4CacheTestCase {
         CqServiceVsdStats cqServiceVsdStats = null;
         try {
           cqServiceVsdStats =
-              ((CqServiceImpl) ((DefaultQueryService) qService).getCqService()).stats;
+              ((CqServiceImpl) ((DefaultQueryService) qService).getCqService()).stats();
         } catch (CqException e) {
           // TODO Auto-generated catch block
           e.printStackTrace();
@@ -193,12 +190,14 @@ public class CqStatsUsingPoolDUnitTest extends JUnit4CacheTestCase {
           fail("Failed to get CQ Service Stats");
         }
 
-        getCache().getLogger().info("#### CQ Service stats: " + " CQs created: "
-            + cqServiceStats.numCqsCreated() + " CQs active: " + cqServiceStats.numCqsActive()
-            + " CQs stopped: " + cqServiceStats.numCqsStopped() + " CQs closed: "
-            + cqServiceStats.numCqsClosed() + " CQs on Client: " + cqServiceStats.numCqsOnClient()
-            + " CQs on region /root/regionA : " + cqServiceVsdStats.numCqsOnRegion("/root/regionA")
-            + " Clients with CQs: " + cqServiceVsdStats.getNumClientsWithCqs());
+        getCache().getLogger()
+            .info("#### CQ Service stats: " + " CQs created: " + cqServiceStats.numCqsCreated()
+                + " CQs active: " + cqServiceStats.numCqsActive() + " CQs stopped: "
+                + cqServiceStats.numCqsStopped() + " CQs closed: " + cqServiceStats.numCqsClosed()
+                + " CQs on Client: " + cqServiceStats.numCqsOnClient()
+                + " CQs on region /root/regionA : "
+                + cqServiceVsdStats.numCqsOnRegion(GemFireCacheImpl.getInstance(), "/root/regionA")
+                + " Clients with CQs: " + cqServiceVsdStats.getNumClientsWithCqs());
 
 
         // Check for created count.
@@ -231,7 +230,7 @@ public class CqStatsUsingPoolDUnitTest extends JUnit4CacheTestCase {
         // Check for CQs on region.
         if (cqsOnRegion != CqQueryUsingPoolDUnitTest.noTest) {
           assertEquals("Number of CQs on region /root/regionA mismatch", cqsOnRegion,
-              cqServiceVsdStats.numCqsOnRegion("/root/regionA"));
+              cqServiceVsdStats.numCqsOnRegion(GemFireCacheImpl.getInstance(), "/root/regionA"));
         }
 
         // Check for clients with CQs count.

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java
index 66c4c0a..5dd0d24 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollector.java
@@ -15,10 +15,8 @@
 
 package org.apache.geode.cache.lucene.internal.distributed;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.logging.log4j.Logger;
@@ -27,7 +25,7 @@ import org.apache.geode.cache.execute.FunctionException;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.cache.lucene.LuceneQuery;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
 
 /**
@@ -44,16 +42,15 @@ import org.apache.geode.internal.logging.LogService;
  */
 public class TopEntriesFunctionCollector
     implements ResultCollector<TopEntriesCollector, TopEntries> {
-  // Use this instance to perform reduce operation
-  final CollectorManager<TopEntriesCollector> manager;
+  private static final Logger logger = LogService.getLogger();
 
-  final String id;
+  // Use this instance to perform reduce operation
+  private final CollectorManager<TopEntriesCollector> manager;
 
-  // Instance of gemfire cache to check status and other utility methods
-  final private GemFireCacheImpl cache;
-  private static final Logger logger = LogService.getLogger();
+  private final String id;
 
   private final Collection<TopEntriesCollector> subResults = new ArrayList<>();
+
   private TopEntriesCollector mergedResults;
 
   public TopEntriesFunctionCollector() {
@@ -65,8 +62,7 @@ public class TopEntriesFunctionCollector
   }
 
   public TopEntriesFunctionCollector(LuceneFunctionContext<TopEntriesCollector> context,
-      GemFireCacheImpl cache) {
-    this.cache = cache;
+      InternalCache cache) {
     id = cache == null ? String.valueOf(this.hashCode()) : cache.getName();
 
     int limit = context == null ? 0 : context.getLimit();
@@ -115,4 +111,8 @@ public class TopEntriesFunctionCollector
       subResults.add(resultOfSingleExecution);
     }
   }
+
+  String id() {
+    return this.id;
+  }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunctionJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunctionJUnitTest.java
index 5313ced..6690850 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunctionJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/LuceneQueryFunctionJUnitTest.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.cache.lucene.internal.distributed;
 
 import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.DEFAULT_FIELD;
@@ -41,6 +40,7 @@ import org.apache.geode.cache.lucene.internal.StringQueryProvider;
 import org.apache.geode.cache.lucene.internal.repository.IndexRepository;
 import org.apache.geode.cache.lucene.internal.repository.IndexResultCollector;
 import org.apache.geode.cache.lucene.internal.repository.RepositoryManager;
+import org.apache.geode.cache.lucene.test.LuceneTestUtilities;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
 import org.apache.geode.internal.cache.execute.InternalRegionFunctionContext;
@@ -58,30 +58,30 @@ import org.mockito.Mockito;
 @Category(UnitTest.class)
 public class LuceneQueryFunctionJUnitTest {
 
-  String regionPath = "/region";
-  String indexName = "index";
-  final EntryScore<String> r1_1 = new EntryScore<String>("key-1-1", .5f);
-  final EntryScore<String> r1_2 = new EntryScore<String>("key-1-2", .4f);
-  final EntryScore<String> r1_3 = new EntryScore<String>("key-1-3", .3f);
-  final EntryScore<String> r2_1 = new EntryScore<String>("key-2-1", .45f);
-  final EntryScore<String> r2_2 = new EntryScore<String>("key-2-2", .35f);
-
-  InternalRegionFunctionContext mockContext;
-  ResultSender<TopEntriesCollector> mockResultSender;
-  Region<Object, Object> mockRegion;
-
-  RepositoryManager mockRepoManager;
-  IndexRepository mockRepository1;
-  IndexRepository mockRepository2;
-  IndexResultCollector mockCollector;
-  InternalLuceneService mockService;
-  LuceneIndexImpl mockIndex;
-  LuceneIndexStats mockStats;
-
-  ArrayList<IndexRepository> repos;
-  LuceneFunctionContext<IndexResultCollector> searchArgs;
-  LuceneQueryProvider queryProvider;
-  Query query;
+  private String regionPath = "/region";
+
+  private final EntryScore<String> r1_1 = new EntryScore<>("key-1-1", .5f);
+  private final EntryScore<String> r1_2 = new EntryScore<>("key-1-2", .4f);
+  private final EntryScore<String> r1_3 = new EntryScore<>("key-1-3", .3f);
+  private final EntryScore<String> r2_1 = new EntryScore<>("key-2-1", .45f);
+  private final EntryScore<String> r2_2 = new EntryScore<>("key-2-2", .35f);
+
+  private InternalRegionFunctionContext mockContext;
+  private ResultSender<TopEntriesCollector> mockResultSender;
+  private Region<Object, Object> mockRegion;
+
+  private RepositoryManager mockRepoManager;
+  private IndexRepository mockRepository1;
+  private IndexRepository mockRepository2;
+  private IndexResultCollector mockCollector;
+  private InternalLuceneService mockService;
+  private LuceneIndexImpl mockIndex;
+  private LuceneIndexStats mockStats;
+
+  private ArrayList<IndexRepository> repos;
+  private LuceneFunctionContext<IndexResultCollector> searchArgs;
+  private LuceneQueryProvider queryProvider;
+  private Query query;
 
   private InternalCache mockCache;
 
@@ -120,7 +120,7 @@ public class LuceneQueryFunctionJUnitTest {
 
     List<EntryScore> hits = result.getEntries().getHits();
     assertEquals(5, hits.size());
-    TopEntriesJUnitTest.verifyResultOrder(result.getEntries().getHits(), r1_1, r2_1, r1_2, r2_2,
+    LuceneTestUtilities.verifyResultOrder(result.getEntries().getHits(), r1_1, r2_1, r1_2, r2_2,
         r1_3);
   }
 
@@ -161,7 +161,7 @@ public class LuceneQueryFunctionJUnitTest {
 
     List<EntryScore> hits = result.getEntries().getHits();
     assertEquals(3, hits.size());
-    TopEntriesJUnitTest.verifyResultOrder(result.getEntries().getHits(), r1_1, r2_1, r1_2);
+    LuceneTestUtilities.verifyResultOrder(result.getEntries().getHits(), r1_1, r2_1, r1_2);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
index 3bfebdf..5767390 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
@@ -26,21 +26,22 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.CopyHelper;
 import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
 import org.apache.geode.cache.lucene.internal.distributed.TopEntriesCollectorManager.ListScanner;
+import org.apache.geode.cache.lucene.test.LuceneTestUtilities;
 import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class TopEntriesCollectorJUnitTest {
 
-  private EntryScore<String> r1_1 = new EntryScore<String>("1-1", .9f);
-  private EntryScore<String> r1_2 = new EntryScore<String>("1-2", .7f);
-  private EntryScore<String> r1_3 = new EntryScore<String>("1-3", .5f);
+  private EntryScore<String> r1_1 = new EntryScore<>("1-1", .9f);
+  private EntryScore<String> r1_2 = new EntryScore<>("1-2", .7f);
+  private EntryScore<String> r1_3 = new EntryScore<>("1-3", .5f);
 
-  private EntryScore<String> r2_1 = new EntryScore<String>("2-1", .85f);
-  private EntryScore<String> r2_2 = new EntryScore<String>("2-2", .65f);
+  private EntryScore<String> r2_1 = new EntryScore<>("2-1", .85f);
+  private EntryScore<String> r2_2 = new EntryScore<>("2-2", .65f);
 
-  private EntryScore<String> r3_1 = new EntryScore<String>("3-1", .8f);
-  private EntryScore<String> r3_2 = new EntryScore<String>("3-2", .6f);
-  private EntryScore<String> r3_3 = new EntryScore<String>("3-3", .4f);
+  private EntryScore<String> r3_1 = new EntryScore<>("3-1", .8f);
+  private EntryScore<String> r3_2 = new EntryScore<>("3-2", .6f);
+  private EntryScore<String> r3_3 = new EntryScore<>("3-3", .4f);
 
   private TopEntriesCollectorManager manager;
 
@@ -72,7 +73,7 @@ public class TopEntriesCollectorJUnitTest {
 
     TopEntriesCollector hits = manager.reduce(collectors);
     assertEquals(8, hits.getEntries().getHits().size());
-    TopEntriesJUnitTest.verifyResultOrder(hits.getEntries().getHits(), r1_1, r2_1, r3_1, r1_2, r2_2,
+    LuceneTestUtilities.verifyResultOrder(hits.getEntries().getHits(), r1_1, r2_1, r3_1, r1_2, r2_2,
         r3_2, r1_3, r3_3);
 
     // input collector should not change
@@ -116,7 +117,7 @@ public class TopEntriesCollectorJUnitTest {
     c1.collect(r1_3.getKey(), r1_3.getScore());
 
     assertEquals(3, c1.getEntries().getHits().size());
-    TopEntriesJUnitTest.verifyResultOrder(c1.getEntries().getHits(), r1_1, r1_2, r1_3);
+    LuceneTestUtilities.verifyResultOrder(c1.getEntries().getHits(), r1_1, r1_2, r1_3);
 
     ListScanner scanner = new ListScanner(c1.getEntries().getHits());
     assertTrue(scanner.hasNext());
@@ -131,6 +132,6 @@ public class TopEntriesCollectorJUnitTest {
     assertFalse(scanner.hasNext());
 
     assertEquals(3, c1.getEntries().getHits().size());
-    TopEntriesJUnitTest.verifyResultOrder(c1.getEntries().getHits(), r1_1, r1_2, r1_3);
+    LuceneTestUtilities.verifyResultOrder(c1.getEntries().getHits(), r1_1, r1_2, r1_3);
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
index bf08877..5fd9e2d 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.cache.lucene.internal.distributed;
 
 import static org.junit.Assert.*;
@@ -20,9 +19,7 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.*;
 
 import java.util.Collection;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
 
 import org.junit.Assert;
 import org.junit.Before;
@@ -31,25 +28,26 @@ import org.junit.experimental.categories.Category;
 import org.mockito.ArgumentMatcher;
 import org.mockito.Mockito;
 
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.execute.FunctionException;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.cache.lucene.test.LuceneTestUtilities;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class TopEntriesFunctionCollectorJUnitTest {
-  EntryScore<String> r1_1;
-  EntryScore<String> r1_2;
-  EntryScore<String> r2_1;
-  EntryScore<String> r2_2;
-  TopEntriesCollector result1, result2;
+
+  private EntryScore<String> r1_1;
+  private EntryScore<String> r1_2;
+  private EntryScore<String> r2_1;
+  private EntryScore<String> r2_2;
+  private TopEntriesCollector result1;
+  private TopEntriesCollector result2;
 
   @Before
   public void initializeCommonObjects() {
-    r1_1 = new EntryScore<String>("3", .9f);
-    r1_2 = new EntryScore<String>("1", .8f);
-    r2_1 = new EntryScore<String>("2", 0.85f);
-    r2_2 = new EntryScore<String>("4", 0.1f);
+    r1_1 = new EntryScore<>("3", .9f);
+    r1_2 = new EntryScore<>("1", .8f);
+    r2_1 = new EntryScore<>("2", 0.85f);
+    r2_2 = new EntryScore<>("4", 0.1f);
 
     result1 = new TopEntriesCollector(null);
     result1.collect(r1_1);
@@ -73,13 +71,9 @@ public class TopEntriesFunctionCollectorJUnitTest {
     collector.addResult(null, result1);
     collector.addResult(null, result2);
 
-    final CountDownLatch insideThread = new CountDownLatch(1);
-    final CountDownLatch resultReceived = new CountDownLatch(1);
-
-    final AtomicReference<TopEntries> result = new AtomicReference<>();
     TopEntries merged = collector.getResult(1, TimeUnit.SECONDS);
     assertEquals(4, merged.size());
-    TopEntriesJUnitTest.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2, r2_2);
+    LuceneTestUtilities.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2, r2_2);
   }
 
   @Test
@@ -95,7 +89,7 @@ public class TopEntriesFunctionCollectorJUnitTest {
     TopEntries merged = collector.getResult();
     Assert.assertNotNull(merged);
     assertEquals(3, merged.size());
-    TopEntriesJUnitTest.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2);
+    LuceneTestUtilities.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2);
   }
 
   @Test
@@ -108,7 +102,7 @@ public class TopEntriesFunctionCollectorJUnitTest {
     TopEntries merged = collector.getResult();
     Assert.assertNotNull(merged);
     assertEquals(4, merged.size());
-    TopEntriesJUnitTest.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2, r2_2);
+    LuceneTestUtilities.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2, r2_2);
   }
 
   @Test
@@ -121,12 +115,12 @@ public class TopEntriesFunctionCollectorJUnitTest {
     TopEntries merged = collector.getResult();
     Assert.assertNotNull(merged);
     assertEquals(4, merged.size());
-    TopEntriesJUnitTest.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2, r2_2);
+    LuceneTestUtilities.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2, r2_2);
 
     merged = collector.getResult();
     Assert.assertNotNull(merged);
     assertEquals(4, merged.size());
-    TopEntriesJUnitTest.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2, r2_2);
+    LuceneTestUtilities.verifyResultOrder(merged.getHits(), r1_1, r2_1, r1_2, r2_2);
   }
 
   @Test
@@ -167,7 +161,7 @@ public class TopEntriesFunctionCollectorJUnitTest {
     TopEntries merged = collector.getResult();
     Assert.assertNotNull(merged);
     assertEquals(2, merged.size());
-    TopEntriesJUnitTest.verifyResultOrder(merged.getHits(), r2_1, r2_2);
+    LuceneTestUtilities.verifyResultOrder(merged.getHits(), r2_1, r2_2);
   }
 
   @Test(expected = RuntimeException.class)
@@ -184,10 +178,10 @@ public class TopEntriesFunctionCollectorJUnitTest {
 
   @Test
   public void testCollectorName() {
-    GemFireCacheImpl mockCache = mock(GemFireCacheImpl.class);
+    InternalCache mockCache = mock(InternalCache.class);
     Mockito.doReturn("server").when(mockCache).getName();
 
     TopEntriesFunctionCollector function = new TopEntriesFunctionCollector(null, mockCache);
-    assertEquals("server", function.id);
+    assertEquals("server", function.id());
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
index fcfebbc..e21ac7f 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
@@ -16,9 +16,6 @@ package org.apache.geode.cache.lucene.internal.distributed;
 
 import static org.junit.Assert.*;
 
-import java.util.Collection;
-import java.util.Iterator;
-
 import org.jmock.Mockery;
 import org.jmock.lib.concurrent.Synchroniser;
 import org.jmock.lib.legacy.ClassImposteriser;
@@ -30,6 +27,7 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.CopyHelper;
 import org.apache.geode.cache.lucene.LuceneQueryFactory;
 import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
+import org.apache.geode.cache.lucene.test.LuceneTestUtilities;
 import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -44,34 +42,34 @@ public class TopEntriesJUnitTest {
 
   @Test
   public void testPopulateTopEntries() {
-    TopEntries<String> hits = new TopEntries<String>();
+    TopEntries<String> hits = new TopEntries<>();
     hits.addHit(r1_1);
     hits.addHit(r2_1);
     hits.addHit(r1_2);
     hits.addHit(r2_2);
 
     assertEquals(4, hits.size());
-    verifyResultOrder(hits.getHits(), r1_1, r2_1, r1_2, r2_2);
+    LuceneTestUtilities.verifyResultOrder(hits.getHits(), r1_1, r2_1, r1_2, r2_2);
   }
 
   @Test
   public void putSameScoreEntries() {
-    TopEntries<String> hits = new TopEntries<String>();
-    EntryScore<String> r1 = new EntryScore<String>("1", .8f);
-    EntryScore<String> r2 = new EntryScore<String>("2", .8f);
+    TopEntries<String> hits = new TopEntries<>();
+    EntryScore<String> r1 = new EntryScore<>("1", .8f);
+    EntryScore<String> r2 = new EntryScore<>("2", .8f);
     hits.addHit(r1);
     hits.addHit(r2);
 
     assertEquals(2, hits.size());
-    verifyResultOrder(hits.getHits(), r1, r2);
+    LuceneTestUtilities.verifyResultOrder(hits.getHits(), r1, r2);
   }
 
   @Test
   public void testInitialization() {
-    TopEntries<String> hits = new TopEntries<String>();
+    TopEntries<String> hits = new TopEntries<>();
     assertEquals(LuceneQueryFactory.DEFAULT_LIMIT, hits.getLimit());
 
-    hits = new TopEntries<String>(123);
+    hits = new TopEntries<>(123);
     assertEquals(123, hits.getLimit());
   }
 
@@ -82,47 +80,33 @@ public class TopEntriesJUnitTest {
 
   @Test
   public void enforceLimit() throws Exception {
-    TopEntries<String> hits = new TopEntries<String>(3);
+    TopEntries<String> hits = new TopEntries<>(3);
     hits.addHit(r1_1);
     hits.addHit(r2_1);
     hits.addHit(r1_2);
     hits.addHit(r2_2);
 
     assertEquals(3, hits.size());
-    verifyResultOrder(hits.getHits(), r1_1, r2_1, r1_2);
+    LuceneTestUtilities.verifyResultOrder(hits.getHits(), r1_1, r2_1, r1_2);
   }
 
   @Test
   public void testSerialization() {
     LuceneServiceImpl.registerDataSerializables();
-    TopEntries<String> hits = new TopEntries<String>(3);
+    TopEntries<String> hits = new TopEntries<>(3);
 
     TopEntries<String> copy = CopyHelper.deepCopy(hits);
     assertEquals(3, copy.getLimit());
     assertEquals(0, copy.getHits().size());
 
-    hits = new TopEntries<String>(3);
+    hits = new TopEntries<>(3);
     hits.addHit(r1_1);
     hits.addHit(r2_1);
     hits.addHit(r1_2);
 
     copy = CopyHelper.deepCopy(hits);
     assertEquals(3, copy.size());
-    verifyResultOrder(copy.getHits(), r1_1, r2_1, r1_2);
-  }
-
-  // TODO: extract to lucene test util class
-  public static void verifyResultOrder(Collection<EntryScore<String>> list,
-      EntryScore<String>... expectedEntries) {
-    Iterator<EntryScore<String>> iter = list.iterator();
-    for (EntryScore expectedEntry : expectedEntries) {
-      if (!iter.hasNext()) {
-        fail();
-      }
-      EntryScore toVerify = iter.next();
-      assertEquals(expectedEntry.getKey(), toVerify.getKey());
-      assertEquals(expectedEntry.getScore(), toVerify.getScore(), .0f);
-    }
+    LuceneTestUtilities.verifyResultOrder(copy.getHits(), r1_1, r2_1, r1_2);
   }
 
   @Before

http://git-wip-us.apache.org/repos/asf/geode/blob/363e50d2/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
index 5563112..17f4dea 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
@@ -18,8 +18,10 @@ import static org.junit.Assert.*;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -44,6 +46,7 @@ import org.apache.geode.cache.lucene.LuceneService;
 import org.apache.geode.cache.lucene.LuceneServiceProvider;
 import org.apache.geode.cache.lucene.internal.LuceneIndexForPartitionedRegion;
 import org.apache.geode.cache.lucene.internal.LuceneServiceImpl;
+import org.apache.geode.cache.lucene.internal.distributed.EntryScore;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
 import org.apache.geode.test.dunit.VM;
@@ -82,6 +85,19 @@ public class LuceneTestUtilities {
   public static String Quarter3 = "Q3";
   public static String Quarter4 = "Q4";
 
+  public static void verifyResultOrder(Collection<EntryScore<String>> list,
+      EntryScore<String>... expectedEntries) {
+    Iterator<EntryScore<String>> iter = list.iterator();
+    for (EntryScore expectedEntry : expectedEntries) {
+      if (!iter.hasNext()) {
+        fail();
+      }
+      EntryScore toVerify = iter.next();
+      assertEquals(expectedEntry.getKey(), toVerify.getKey());
+      assertEquals(expectedEntry.getScore(), toVerify.getScore(), .0f);
+    }
+  }
+
   public static class IntRangeQueryProvider implements LuceneQueryProvider {
     String fieldName;
     int lowerValue;


[10/13] geode git commit: Safe refactorings

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/TXWriterTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/TXWriterTestCase.java b/geode-core/src/test/java/org/apache/geode/TXWriterTestCase.java
index 987f22f..a75e167 100644
--- a/geode-core/src/test/java/org/apache/geode/TXWriterTestCase.java
+++ b/geode-core/src/test/java/org/apache/geode/TXWriterTestCase.java
@@ -14,30 +14,43 @@
  */
 package org.apache.geode;
 
-import org.apache.geode.cache.*;
-import org.apache.geode.cache.util.CacheListenerAdapter;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+
+import java.util.Properties;
+
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 
-import java.util.Properties;
-
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import org.apache.geode.cache.AttributesFactory;
+import org.apache.geode.cache.AttributesMutator;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheException;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.CacheTransactionManager;
+import org.apache.geode.cache.CacheWriter;
+import org.apache.geode.cache.CacheWriterException;
+import org.apache.geode.cache.EntryEvent;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionEvent;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.cache.TransactionEvent;
+import org.apache.geode.cache.TransactionListener;
+import org.apache.geode.cache.util.CacheListenerAdapter;
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
 
 /**
  * Extracted from TXWriterJUnitTest to share with TXWriterOOMEJUnitTest.
- * 
  */
 @SuppressWarnings("deprecation")
 public class TXWriterTestCase {
 
-  protected int cbCount;
-  protected int failedCommits = 0;
-  protected int afterCommits = 0;
-  protected int afterRollbacks = 0;
+  int cbCount;
+  int failedCommits = 0;
+  int afterCommits = 0;
+  int afterRollbacks = 0;
 
   protected GemFireCacheImpl cache;
   protected CacheTransactionManager txMgr;
@@ -46,10 +59,13 @@ public class TXWriterTestCase {
   protected void createCache() throws CacheException {
     Properties p = new Properties();
     p.setProperty(MCAST_PORT, "0"); // loner
+
     this.cache = (GemFireCacheImpl) CacheFactory.create(DistributedSystem.connect(p));
-    AttributesFactory<?, ?> af = new AttributesFactory<String, String>();
+
+    AttributesFactory<String, String> af = new AttributesFactory<>();
     af.setScope(Scope.DISTRIBUTED_NO_ACK);
     af.setIndexMaintenanceSynchronous(true);
+
     this.region = this.cache.createRegion("TXTest", af.create());
     this.txMgr = this.cache.getCacheTransactionManager();
   }
@@ -95,79 +111,96 @@ public class TXWriterTestCase {
     }
   }
 
-  protected void installCacheListenerAndWriter() {
+  void installCacheListenerAndWriter() {
     AttributesMutator<String, String> mutator = this.region.getAttributesMutator();
     mutator.setCacheListener(new CacheListenerAdapter<String, String>() {
+      @Override
       public void close() {
         cbCount++;
       }
 
+      @Override
       public void afterCreate(EntryEvent<String, String> event) {
         cbCount++;
       }
 
+      @Override
       public void afterUpdate(EntryEvent<String, String> event) {
         cbCount++;
       }
 
+      @Override
       public void afterInvalidate(EntryEvent<String, String> event) {
         cbCount++;
       }
 
+      @Override
       public void afterDestroy(EntryEvent<String, String> event) {
         cbCount++;
       }
 
+      @Override
       public void afterRegionInvalidate(RegionEvent<String, String> event) {
         cbCount++;
       }
 
+      @Override
       public void afterRegionDestroy(RegionEvent<String, String> event) {
         cbCount++;
       }
     });
     mutator.setCacheWriter(new CacheWriter<String, String>() {
+      @Override
       public void close() {
         cbCount++;
       }
 
+      @Override
       public void beforeUpdate(EntryEvent<String, String> event) throws CacheWriterException {
         cbCount++;
       }
 
+      @Override
       public void beforeCreate(EntryEvent<String, String> event) throws CacheWriterException {
         cbCount++;
       }
 
+      @Override
       public void beforeDestroy(EntryEvent<String, String> event) throws CacheWriterException {
         cbCount++;
       }
 
+      @Override
       public void beforeRegionDestroy(RegionEvent<String, String> event)
           throws CacheWriterException {
         cbCount++;
       }
 
+      @Override
       public void beforeRegionClear(RegionEvent<String, String> event) throws CacheWriterException {
         cbCount++;
       }
     });
   }
 
-  protected void installTransactionListener() {
-    ((CacheTransactionManager) this.txMgr).setListener(new TransactionListener() {
+  void installTransactionListener() {
+    this.txMgr.setListener(new TransactionListener() {
+      @Override
       public void afterFailedCommit(TransactionEvent event) {
         failedCommits++;
       }
 
+      @Override
       public void afterCommit(TransactionEvent event) {
         afterCommits++;
       }
 
+      @Override
       public void afterRollback(TransactionEvent event) {
         afterRollbacks++;
       }
 
+      @Override
       public void close() {}
     });
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
index 903b212..15cfaa7 100755
--- a/geode-core/src/test/java/org/apache/geode/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
@@ -946,8 +946,7 @@ public class ResourceManagerWithQueryMonitorDUnitTest extends ClientServerTestCa
     server.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
-        cache.TEST_MAX_QUERY_EXECUTION_TIME_OVERRIDE_EXCEPTION = false;
-        cache.TEST_MAX_QUERY_EXECUTION_TIME = -1;
+        cache.testMaxQueryExecutionTime = -1;
         return null;
       }
     });
@@ -972,11 +971,9 @@ public class ResourceManagerWithQueryMonitorDUnitTest extends ClientServerTestCa
         GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
 
         if (queryTimeout != -1) {
-          cache.TEST_MAX_QUERY_EXECUTION_TIME_OVERRIDE_EXCEPTION = true;
-          cache.TEST_MAX_QUERY_EXECUTION_TIME = queryTimeout;
+          cache.testMaxQueryExecutionTime = queryTimeout;
         } else {
-          cache.TEST_MAX_QUERY_EXECUTION_TIME_OVERRIDE_EXCEPTION = false;
-          cache.TEST_MAX_QUERY_EXECUTION_TIME = -1;
+          cache.testMaxQueryExecutionTime = -1;
         }
 
         if (criticalThreshold != 0) {
@@ -1074,7 +1071,7 @@ public class ResourceManagerWithQueryMonitorDUnitTest extends ClientServerTestCa
   private boolean isExceptionDueToTimeout(QueryException e, long queryTimeout) {
     String message = e.getMessage();
     // -1 needs to be matched due to client/server set up, BaseCommand uses the
-    // MAX_QUERY_EXECUTION_TIME and not the TEST_MAX_QUERY_EXECUTION_TIME
+    // MAX_QUERY_EXECUTION_TIME and not the testMaxQueryExecutionTime
     return (message.contains("The QueryMonitor thread may be sleeping longer than")
         || message.contains(LocalizedStrings.QueryMonitor_LONG_RUNNING_QUERY_CANCELED
             .toLocalizedString(queryTimeout))

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/disttx/DistTXDebugDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/disttx/DistTXDebugDUnitTest.java b/geode-core/src/test/java/org/apache/geode/disttx/DistTXDebugDUnitTest.java
index 0d2f2b6..e72823e 100644
--- a/geode-core/src/test/java/org/apache/geode/disttx/DistTXDebugDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/disttx/DistTXDebugDUnitTest.java
@@ -46,7 +46,6 @@ import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -60,7 +59,7 @@ public class DistTXDebugDUnitTest extends JUnit4CacheTestCase {
   protected VM accessor = null;
   protected VM dataStore1 = null;
   protected VM dataStore2 = null;
-  protected VM dataStore3 = null;
+  private VM dataStore3 = null;
 
   @Override
   public final void postSetUp() throws Exception {
@@ -106,30 +105,28 @@ public class DistTXDebugDUnitTest extends JUnit4CacheTestCase {
   public static void createPR(String partitionedRegionName, Integer redundancy,
       Integer localMaxMemory, Integer totalNumBuckets, Object colocatedWith,
       Boolean isPartitionResolver, Boolean concurrencyChecks) {
-    PartitionAttributesFactory paf = new PartitionAttributesFactory();
-
-    paf.setRedundantCopies(redundancy.intValue());
+    PartitionAttributesFactory<String, String> paf = new PartitionAttributesFactory();
+    paf.setRedundantCopies(redundancy);
     if (localMaxMemory != null) {
-      paf.setLocalMaxMemory(localMaxMemory.intValue());
+      paf.setLocalMaxMemory(localMaxMemory);
     }
     if (totalNumBuckets != null) {
-      paf.setTotalNumBuckets(totalNumBuckets.intValue());
+      paf.setTotalNumBuckets(totalNumBuckets);
     }
     if (colocatedWith != null) {
       paf.setColocatedWith((String) colocatedWith);
     }
-    if (isPartitionResolver.booleanValue()) {
+    if (isPartitionResolver) {
       paf.setPartitionResolver(new CustomerIDPartitionResolver("CustomerIDPartitionResolver"));
     }
-    PartitionAttributes prAttr = paf.create();
-    AttributesFactory attr = new AttributesFactory();
+    PartitionAttributes<String, String> prAttr = paf.create();
+
+    AttributesFactory<String, String> attr = new AttributesFactory();
     attr.setPartitionAttributes(prAttr);
     attr.setConcurrencyChecksEnabled(concurrencyChecks);
-    // assertNotNull(basicGetCache());
-    // Region pr = basicGetCache().createRegion(partitionedRegionName,
-    // attr.create());
+
     assertNotNull(basicGetCache());
-    Region pr = basicGetCache().createRegion(partitionedRegionName, attr.create());
+    Region<String, String> pr = basicGetCache().createRegion(partitionedRegionName, attr.create());
     assertNotNull(pr);
     LogWriterUtils.getLogWriter().info(
         "Partitioned Region " + partitionedRegionName + " created Successfully :" + pr.toString());
@@ -912,55 +909,54 @@ public class DistTXDebugDUnitTest extends JUnit4CacheTestCase {
   public void testTXRR2_dataNodeAsCoordinator() throws Exception {
     performTXRRtestOps(true);
   }
-}
-
 
-class DummyKeyBasedRoutingResolver implements PartitionResolver, DataSerializable {
-  Integer dummyID;
+  private static class DummyKeyBasedRoutingResolver implements PartitionResolver, DataSerializable {
+    Integer dummyID;
 
-  public DummyKeyBasedRoutingResolver() {}
+    public DummyKeyBasedRoutingResolver() {}
 
-  public DummyKeyBasedRoutingResolver(int id) {
-    this.dummyID = new Integer(id);
-  }
+    public DummyKeyBasedRoutingResolver(int id) {
+      this.dummyID = new Integer(id);
+    }
 
-  public String getName() {
-    // TODO Auto-generated method stub
-    return null;
-  }
+    public String getName() {
+      // TODO Auto-generated method stub
+      return null;
+    }
 
-  public Serializable getRoutingObject(EntryOperation opDetails) {
-    return (Serializable) opDetails.getKey();
-  }
+    public Serializable getRoutingObject(EntryOperation opDetails) {
+      return (Serializable) opDetails.getKey();
+    }
 
-  public void close() {
-    // TODO Auto-generated method stub
-  }
+    public void close() {
+      // TODO Auto-generated method stub
+    }
 
-  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    this.dummyID = DataSerializer.readInteger(in);
-  }
+    public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+      this.dummyID = DataSerializer.readInteger(in);
+    }
 
-  public void toData(DataOutput out) throws IOException {
-    DataSerializer.writeInteger(this.dummyID, out);
-  }
+    public void toData(DataOutput out) throws IOException {
+      DataSerializer.writeInteger(this.dummyID, out);
+    }
 
-  @Override
-  public int hashCode() {
-    int i = this.dummyID.intValue();
-    return i;
-  }
+    @Override
+    public int hashCode() {
+      int i = this.dummyID.intValue();
+      return i;
+    }
 
-  @Override
-  public boolean equals(Object o) {
-    if (this == o)
-      return true;
+    @Override
+    public boolean equals(Object o) {
+      if (this == o)
+        return true;
 
-    if (!(o instanceof DummyKeyBasedRoutingResolver))
-      return false;
+      if (!(o instanceof DummyKeyBasedRoutingResolver))
+        return false;
 
-    DummyKeyBasedRoutingResolver otherDummyID = (DummyKeyBasedRoutingResolver) o;
-    return (otherDummyID.dummyID.equals(dummyID));
+      DummyKeyBasedRoutingResolver otherDummyID = (DummyKeyBasedRoutingResolver) o;
+      return (otherDummyID.dummyID.equals(dummyID));
 
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/disttx/DistTXJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/disttx/DistTXJUnitTest.java b/geode-core/src/test/java/org/apache/geode/disttx/DistTXJUnitTest.java
index 8abccc6..754b554 100644
--- a/geode-core/src/test/java/org/apache/geode/disttx/DistTXJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/disttx/DistTXJUnitTest.java
@@ -14,6 +14,15 @@
  */
 package org.apache.geode.disttx;
 
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.junit.Assert.*;
+
+import java.util.Properties;
+
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.TXJUnitTest;
 import org.apache.geode.cache.CacheException;
 import org.apache.geode.cache.CacheFactory;
@@ -21,51 +30,33 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.test.junit.categories.DistributedTransactionsTest;
 import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.util.Properties;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
 
 /**
  * Run the basic transaction functionality tests in TXJUnitTest after setting
  * "distributed-transactions" property to true
- *
  */
 @Category({IntegrationTest.class, DistributedTransactionsTest.class})
 public class DistTXJUnitTest extends TXJUnitTest {
 
-  public DistTXJUnitTest() {}
-
   @Override
   protected void createCache() throws Exception {
     Properties p = new Properties();
     p.setProperty(MCAST_PORT, "0"); // loner
     p.setProperty(DISTRIBUTED_TRANSACTIONS, "true");
+
     this.cache = (GemFireCacheImpl) CacheFactory.create(DistributedSystem.connect(p));
+
     createRegion();
     this.txMgr = this.cache.getCacheTransactionManager();
-    assert (this.txMgr.isDistributed());
+
+    assertTrue(this.txMgr.isDistributed());
+
     this.listenerAfterCommit = 0;
     this.listenerAfterFailedCommit = 0;
     this.listenerAfterRollback = 0;
     this.listenerClose = 0;
   }
 
-  @Before
-  public void setUp() throws Exception {
-    createCache();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    closeCache();
-  }
-
   @Override
   @Test
   @Ignore

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/disttx/DistTXPersistentDebugDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/disttx/DistTXPersistentDebugDUnitTest.java b/geode-core/src/test/java/org/apache/geode/disttx/DistTXPersistentDebugDUnitTest.java
index 5753f5c..d999da9 100644
--- a/geode-core/src/test/java/org/apache/geode/disttx/DistTXPersistentDebugDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/disttx/DistTXPersistentDebugDUnitTest.java
@@ -16,6 +16,9 @@ package org.apache.geode.disttx;
 
 import static org.apache.geode.test.dunit.Assert.*;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheTransactionManager;
@@ -31,8 +34,6 @@ import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 @Category(DistributedTest.class)
 public class DistTXPersistentDebugDUnitTest extends DistTXDebugDUnitTest {
@@ -61,7 +62,7 @@ public class DistTXPersistentDebugDUnitTest extends DistTXDebugDUnitTest {
     });
   }
 
-  protected void createPesistentPR(Object[] attributes) {
+  private void createPersistentPR(Object[] attributes) {
     dataStore1.invoke(DistTXPersistentDebugDUnitTest.class, "createPersistentPR", attributes);
     dataStore2.invoke(DistTXPersistentDebugDUnitTest.class, "createPersistentPR", attributes);
     // dataStore3.invoke(TxPersistentDebugDUnit.class, "createPR", attributes);
@@ -76,7 +77,7 @@ public class DistTXPersistentDebugDUnitTest extends DistTXDebugDUnitTest {
         getPersistentPRAttributes(1, -1, basicGetCache(), 113, true));
   }
 
-  protected static RegionAttributes getPersistentPRAttributes(final int redundancy,
+  private static RegionAttributes getPersistentPRAttributes(final int redundancy,
       final int recoveryDelay, Cache cache, int numBuckets, boolean synchronous) {
     DiskStore ds = cache.findDiskStore("disk");
     if (ds == null) {
@@ -92,8 +93,7 @@ public class DistTXPersistentDebugDUnitTest extends DistTXDebugDUnitTest {
     af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
     af.setDiskStoreName("disk");
     af.setDiskSynchronous(synchronous);
-    RegionAttributes attr = af.create();
-    return attr;
+    return af.create();
   }
 
   @Test
@@ -101,7 +101,8 @@ public class DistTXPersistentDebugDUnitTest extends DistTXDebugDUnitTest {
     createCacheInAllVms();
     final String regionName = "persistentCustomerPRRegion";
     Object[] attrs = new Object[] {regionName};
-    createPesistentPR(attrs);
+    createPersistentPR(attrs);
+
     SerializableCallable TxOps = new SerializableCallable() {
       @Override
       public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterJUnitTest.java b/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterJUnitTest.java
index 0a61b1f..dec2f88 100644
--- a/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterJUnitTest.java
@@ -14,6 +14,13 @@
  */
 package org.apache.geode.disttx;
 
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.junit.Assert.*;
+
+import java.util.Properties;
+
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.TXWriterJUnitTest;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.CacheException;
@@ -24,11 +31,6 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.test.junit.categories.DistributedTransactionsTest;
 import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.junit.experimental.categories.Category;
-
-import java.util.Properties;
-
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 
 /**
  * Same tests as that of {@link TXWriterJUnitTest} after setting "distributed-transactions" property
@@ -37,19 +39,22 @@ import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 @Category({IntegrationTest.class, DistributedTransactionsTest.class})
 public class DistTXWriterJUnitTest extends TXWriterJUnitTest {
 
-  public DistTXWriterJUnitTest() {}
-
+  @Override
   protected void createCache() throws CacheException {
-    Properties p = new Properties();
-    p.setProperty(MCAST_PORT, "0"); // loner
-    p.setProperty(ConfigurationProperties.DISTRIBUTED_TRANSACTIONS, "true");
-    this.cache = (GemFireCacheImpl) CacheFactory.create(DistributedSystem.connect(p));
-    AttributesFactory<?, ?> af = new AttributesFactory<String, String>();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
-    af.setIndexMaintenanceSynchronous(true);
-    this.region = this.cache.createRegion("TXTest", af.create());
+    Properties properties = new Properties();
+    properties.setProperty(MCAST_PORT, "0"); // loner
+    properties.setProperty(ConfigurationProperties.DISTRIBUTED_TRANSACTIONS, "true");
+
+    this.cache = (GemFireCacheImpl) CacheFactory.create(DistributedSystem.connect(properties));
+
+    AttributesFactory<String, String> attributesFactory = new AttributesFactory<>();
+    attributesFactory.setScope(Scope.DISTRIBUTED_NO_ACK);
+    attributesFactory.setIndexMaintenanceSynchronous(true);
+
+    this.region = this.cache.createRegion("TXTest", attributesFactory.create());
     this.txMgr = this.cache.getCacheTransactionManager();
-    assert (this.txMgr.isDistributed());
+
+    assertTrue(this.txMgr.isDistributed());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterOOMEJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterOOMEJUnitTest.java b/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterOOMEJUnitTest.java
index b99d3fd..896530d 100644
--- a/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterOOMEJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/disttx/DistTXWriterOOMEJUnitTest.java
@@ -14,21 +14,23 @@
  */
 package org.apache.geode.disttx;
 
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.junit.Assert.*;
+
+import java.util.Properties;
+
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.TXWriterOOMEJUnitTest;
 import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.CacheException;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.Scope;
-import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.test.junit.categories.DistributedTransactionsTest;
 import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.junit.experimental.categories.Category;
-
-import java.util.Properties;
-
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 
 /**
  * Same tests as that of {@link TXWriterOOMEJUnitTest} after setting "distributed-transactions"
@@ -37,19 +39,22 @@ import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 @Category({IntegrationTest.class, DistributedTransactionsTest.class})
 public class DistTXWriterOOMEJUnitTest extends TXWriterOOMEJUnitTest {
 
-  public DistTXWriterOOMEJUnitTest() {}
-
+  @Override
   protected void createCache() throws CacheException {
-    Properties p = new Properties();
-    p.setProperty(MCAST_PORT, "0"); // loner
-    p.setProperty(ConfigurationProperties.DISTRIBUTED_TRANSACTIONS, "true");
-    this.cache = (GemFireCacheImpl) CacheFactory.create(DistributedSystem.connect(p));
-    AttributesFactory<?, ?> af = new AttributesFactory<String, String>();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
-    af.setIndexMaintenanceSynchronous(true);
-    this.region = this.cache.createRegion("TXTest", af.create());
+    Properties properties = new Properties();
+    properties.setProperty(MCAST_PORT, "0"); // loner
+    properties.setProperty(ConfigurationProperties.DISTRIBUTED_TRANSACTIONS, "true");
+
+    this.cache = (GemFireCacheImpl) CacheFactory.create(DistributedSystem.connect(properties));
+
+    AttributesFactory<String, String> attributesFactory = new AttributesFactory<>();
+    attributesFactory.setScope(Scope.DISTRIBUTED_NO_ACK);
+    attributesFactory.setIndexMaintenanceSynchronous(true);
+
+    this.region = this.cache.createRegion("TXTest", attributesFactory.create());
     this.txMgr = this.cache.getCacheTransactionManager();
-    assert (this.txMgr.isDistributed());
+
+    assertTrue(this.txMgr.isDistributed());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/disttx/DistributedTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/disttx/DistributedTransactionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/disttx/DistributedTransactionDUnitTest.java
index 5471565..fe79801 100644
--- a/geode-core/src/test/java/org/apache/geode/disttx/DistributedTransactionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/disttx/DistributedTransactionDUnitTest.java
@@ -44,7 +44,6 @@ import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.i18n.LogWriterI18n;
 import org.apache.geode.internal.AvailablePort;
 import org.apache.geode.internal.cache.BucketRegion;
-import org.apache.geode.internal.cache.CacheServerImpl;
 import org.apache.geode.internal.cache.DistTXState;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
@@ -1383,7 +1382,7 @@ public class DistributedTransactionDUnitTest extends JUnit4CacheTestCase {
       @Override
       public Object call() throws Exception {
         CountDownLatch cdl = new CountDownLatch(1);
-        GemFireCacheImpl.internalBeforeApplyChanges = new WaitRelease(cdl, "TX OP");
+        DistTXState.internalBeforeApplyChanges = new WaitRelease(cdl, "TX OP");
         return null;
       }
     };
@@ -1396,7 +1395,7 @@ public class DistributedTransactionDUnitTest extends JUnit4CacheTestCase {
       @Override
       public Object call() throws Exception {
         CountDownLatch cdl = new CountDownLatch(1);
-        GemFireCacheImpl.internalBeforeNonTXBasicPut = new WaitRelease(cdl, "NON TX OP");
+        DistTXState.internalBeforeNonTXBasicPut = new WaitRelease(cdl, "NON TX OP");
         return null;
       }
     };
@@ -1457,7 +1456,7 @@ public class DistributedTransactionDUnitTest extends JUnit4CacheTestCase {
     execute(secondary, new SerializableCallable() {
       @Override
       public Object call() throws Exception {
-        Runnable r = GemFireCacheImpl.internalBeforeNonTXBasicPut;
+        Runnable r = DistTXState.internalBeforeNonTXBasicPut;
         assert (r != null && r instanceof WaitRelease);
         WaitRelease e = (WaitRelease) r;
         e.release();
@@ -1469,7 +1468,7 @@ public class DistributedTransactionDUnitTest extends JUnit4CacheTestCase {
     execute(secondary, new SerializableCallable() {
       @Override
       public Object call() throws Exception {
-        Runnable r = GemFireCacheImpl.internalBeforeApplyChanges;
+        Runnable r = DistTXState.internalBeforeApplyChanges;
         assert (r != null && r instanceof WaitRelease);
         WaitRelease e = (WaitRelease) r;
         e.release();

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/disttx/PRDistTXJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/disttx/PRDistTXJUnitTest.java b/geode-core/src/test/java/org/apache/geode/disttx/PRDistTXJUnitTest.java
index f27c099..268a733 100644
--- a/geode-core/src/test/java/org/apache/geode/disttx/PRDistTXJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/disttx/PRDistTXJUnitTest.java
@@ -14,6 +14,15 @@
  */
 package org.apache.geode.disttx;
 
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.junit.Assert.*;
+
+import java.util.Properties;
+
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.cache.CacheException;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.distributed.ConfigurationProperties;
@@ -22,33 +31,27 @@ import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.PRTXJUnitTest;
 import org.apache.geode.test.junit.categories.DistributedTransactionsTest;
 import org.apache.geode.test.junit.categories.IntegrationTest;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.util.Properties;
-
-import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 
 /**
  * Same tests as that of {@link PRTXJUnitTest} after setting "distributed-transactions" property to
  * true
- *
  */
 @Category({IntegrationTest.class, DistributedTransactionsTest.class})
 public class PRDistTXJUnitTest extends PRTXJUnitTest {
 
-  public PRDistTXJUnitTest() {}
-
   @Override
   protected void createCache() throws Exception {
     Properties p = new Properties();
     p.setProperty(MCAST_PORT, "0"); // loner
     p.setProperty(ConfigurationProperties.DISTRIBUTED_TRANSACTIONS, "true");
+
     this.cache = (GemFireCacheImpl) CacheFactory.create(DistributedSystem.connect(p));
+
     createRegion();
     this.txMgr = this.cache.getCacheTransactionManager();
-    assert (this.txMgr.isDistributed());
+
+    assertTrue(this.txMgr.isDistributed());
+
     this.listenerAfterCommit = 0;
     this.listenerAfterFailedCommit = 0;
     this.listenerAfterRollback = 0;

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/internal/cache/PRTXJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/PRTXJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/PRTXJUnitTest.java
index d2bad64..1caffbd 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/PRTXJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/PRTXJUnitTest.java
@@ -12,9 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-/**
- * 
- */
 package org.apache.geode.internal.cache;
 
 import org.junit.Ignore;
@@ -30,132 +27,83 @@ import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
 import org.apache.geode.cache.query.QueryException;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.InternalRegionArguments;
-import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
-/**
- *
- */
 @Category(IntegrationTest.class)
 public class PRTXJUnitTest extends TXJUnitTest {
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.TXTest#createRegion()
-   */
   @Override
   protected void createRegion() throws Exception {
-    AttributesFactory af = new AttributesFactory();
-    af.setConcurrencyChecksEnabled(false); // test validation expects this behavior
-    af.setPartitionAttributes(new PartitionAttributesFactory().setTotalNumBuckets(3).create());
-    // this.region = this.cache.createRegion("PRTXJUnitTest", af.create());
-    this.region = new PRWithLocalOps("PRTXJUnitTest", af.create(), null, this.cache,
-        new InternalRegionArguments().setDestroyLockFlag(true).setRecreateFlag(false)
+    AttributesFactory attributesFactory = new AttributesFactory();
+    // test validation expects this behavior
+    attributesFactory.setConcurrencyChecksEnabled(false);
+    attributesFactory
+        .setPartitionAttributes(new PartitionAttributesFactory().setTotalNumBuckets(3).create());
+
+    this.region = new PRWithLocalOps(getClass().getSimpleName(), attributesFactory.create(), null,
+        this.cache, new InternalRegionArguments().setDestroyLockFlag(true).setRecreateFlag(false)
             .setSnapshotInputStream(null).setImageTarget(null));
+
     ((PartitionedRegion) this.region).initialize(null, null, null);
     ((PartitionedRegion) this.region).postCreateRegion();
     this.cache.setRegionByPath(this.region.getFullPath(), (LocalRegion) this.region);
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.TXTest#checkUserAttributeConflict(org.apache.geode.internal.cache.
-   * TXManagerImpl)
-   */
   @Override
   protected void checkUserAttributeConflict(CacheTransactionManager txMgrImpl) {}
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see
-   * org.apache.geode.TXTest#checkSubRegionCollecection(org.apache.geode.internal.cache.LocalRegion)
-   */
   @Override
-  protected void checkSubRegionCollecection(Region reg1) {}
+  protected void checkSubRegionCollection(Region reg1) {}
 
   @Override
   @Ignore
   @Test
-  public void testTXAndQueries() throws CacheException, QueryException {
-    // TODO fix this?
-  }
+  public void testTXAndQueries() throws CacheException, QueryException {}
 
   @Override
   @Ignore
   @Test
-  public void testCollections() throws CacheException {
-    // TODO make PR iterators tx aware
-  }
+  public void testCollections() throws CacheException {}
 
   @Override
   @Ignore
   @Test
-  public void testTxAlgebra() throws CacheException {
-    // TODO Auto-generated method stub
-  }
+  public void testTxAlgebra() throws CacheException {}
 
   @Test
   public void testTxId() {
-    AttributesFactory<Integer, String> af = new AttributesFactory<Integer, String>();
-    af.setPartitionAttributes(
+    AttributesFactory<String, Integer> attributesFactory = new AttributesFactory<>();
+    attributesFactory.setPartitionAttributes(
         new PartitionAttributesFactory<String, Integer>().setTotalNumBuckets(2).create());
-    Region<String, Integer> r = this.cache.createRegion("testTxId", af.create());
-    r.put("one", 1);
-    CacheTransactionManager mgr = this.cache.getTxManager();
-    mgr.begin();
-    r.put("two", 2);
-    mgr.getTransactionId();
-    mgr.rollback();
+
+    Region<String, Integer> region =
+        this.cache.createRegion("testTxId", attributesFactory.create());
+    region.put("one", 1);
+
+    CacheTransactionManager txManager = this.cache.getTxManager();
+    txManager.begin();
+    region.put("two", 2);
+    txManager.getTransactionId();
+    txManager.rollback();
   }
 
   private static class PRWithLocalOps extends PartitionedRegion {
 
-    /**
-     * @param regionname
-     * @param ra
-     * @param parentRegion
-     * @param cache
-     * @param internalRegionArgs
-     */
-    public PRWithLocalOps(String regionname, RegionAttributes ra, LocalRegion parentRegion,
+    PRWithLocalOps(String regionName, RegionAttributes ra, LocalRegion parentRegion,
         GemFireCacheImpl cache, InternalRegionArguments internalRegionArgs) {
-      super(regionname, ra, parentRegion, cache, internalRegionArgs);
+      super(regionName, ra, parentRegion, cache, internalRegionArgs);
     }
 
-    /*
-     * (non-Javadoc)
-     * 
-     * @see org.apache.geode.internal.cache.PartitionedRegion#localDestroy(java.lang.Object,
-     * java.lang.Object)
-     */
     @Override
     public void localDestroy(Object key, Object callbackArgument) throws EntryNotFoundException {
       super.destroy(key, callbackArgument);
     }
 
-    /*
-     * (non-Javadoc)
-     * 
-     * @see org.apache.geode.internal.cache.PartitionedRegion#localInvalidate(java.lang.Object,
-     * java.lang.Object)
-     */
     @Override
     public void localInvalidate(Object key, Object callbackArgument) throws EntryNotFoundException {
       super.invalidate(key, callbackArgument);
     }
 
-    /*
-     * (non-Javadoc)
-     * 
-     * @see
-     * org.apache.geode.internal.cache.PartitionedRegion#localInvalidateRegion(java.lang.Object)
-     */
     @Override
     public void localInvalidateRegion(Object callbackArgument) {
       super.invalidateRegion(callbackArgument);

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
index d57ce12..b7ee5c8 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/wan/parallel/ParallelQueueRemovalMessageJUnitTest.java
@@ -14,31 +14,59 @@
  */
 package org.apache.geode.internal.cache.wan.parallel;
 
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import org.apache.geode.CancelCriterion;
-import org.apache.geode.cache.*;
-import org.apache.geode.internal.cache.*;
+import org.apache.geode.cache.AttributesFactory;
+import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.EvictionAction;
+import org.apache.geode.cache.EvictionAttributes;
+import org.apache.geode.cache.PartitionAttributes;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.internal.cache.BucketAdvisor;
+import org.apache.geode.internal.cache.BucketRegionQueue;
+import org.apache.geode.internal.cache.BucketRegionQueueHelper;
+import org.apache.geode.internal.cache.EvictionAttributesImpl;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalRegionArguments;
+import org.apache.geode.internal.cache.KeyInfo;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.PartitionedRegionDataStore;
+import org.apache.geode.internal.cache.PartitionedRegionHelper;
+import org.apache.geode.internal.cache.PartitionedRegionStats;
+import org.apache.geode.internal.cache.ProxyBucketRegion;
+import org.apache.geode.internal.cache.RegionQueue;
 import org.apache.geode.internal.cache.lru.LRUAlgorithm;
 import org.apache.geode.internal.cache.partitioned.RegionAdvisor;
 import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
 import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl;
 import org.apache.geode.test.fake.Fakes;
 import org.apache.geode.test.junit.categories.UnitTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.util.*;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 @Category(UnitTest.class)
 public class ParallelQueueRemovalMessageJUnitTest {
 
+  private static final String GATEWAY_SENDER_ID = "ny";
+  private static final int BUCKET_ID = 85;
+  private static final long KEY = 198;
+
   private GemFireCacheImpl cache;
   private PartitionedRegion queueRegion;
   private AbstractGatewaySender sender;
@@ -46,10 +74,6 @@ public class ParallelQueueRemovalMessageJUnitTest {
   private BucketRegionQueue bucketRegionQueue;
   private BucketRegionQueueHelper bucketRegionQueueHelper;
 
-  private static String GATEWAY_SENDER_ID = "ny";
-  private static int BUCKET_ID = 85;
-  private static long KEY = 198l;
-
   @Before
   public void setUpGemFire() {
     createCache();
@@ -115,10 +139,10 @@ public class ParallelQueueRemovalMessageJUnitTest {
     when(this.queueRegion.getPartitionAttributes()).thenReturn(pa);
     when(this.queueRegion.getDataPolicy()).thenReturn(DataPolicy.PARTITION);
     when(pa.getColocatedWith()).thenReturn(null);
-    ProxyBucketRegion pbr = new ProxyBucketRegion(BUCKET_ID, this.queueRegion, pbrIra); // final
-                                                                                        // classes
-                                                                                        // cannot be
-                                                                                        // mocked
+
+    // final classes cannot be mocked
+    ProxyBucketRegion pbr = new ProxyBucketRegion(BUCKET_ID, this.queueRegion, pbrIra);
+
     when(ba.getProxyBucketRegion()).thenReturn(pbr);
 
     // Create RegionAttributes
@@ -182,11 +206,11 @@ public class ParallelQueueRemovalMessageJUnitTest {
     assertFalse(this.bucketRegionQueue.isInitialized());
 
     // Create a real ConcurrentParallelGatewaySenderQueue
-    ParallelGatewaySenderEventProcessor pgsep = createConcurrentParallelGatewaySenderQueue();
+    ParallelGatewaySenderEventProcessor processor = createConcurrentParallelGatewaySenderQueue();
 
     // Add a mock GatewaySenderEventImpl to the temp queue
     BlockingQueue<GatewaySenderEventImpl> tempQueue =
-        createTempQueueAndAddEvent(pgsep, mock(GatewaySenderEventImpl.class));
+        createTempQueueAndAddEvent(processor, mock(GatewaySenderEventImpl.class));
     assertEquals(1, tempQueue.size());
 
     // Create and process a ParallelQueueRemovalMessage (causes the failedBatchRemovalMessageKeys to
@@ -204,14 +228,14 @@ public class ParallelQueueRemovalMessageJUnitTest {
     assertEquals(0, this.bucketRegionQueue.size());
 
     // Create a real ConcurrentParallelGatewaySenderQueue
-    ParallelGatewaySenderEventProcessor pgsep = createConcurrentParallelGatewaySenderQueue();
+    ParallelGatewaySenderEventProcessor processor = createConcurrentParallelGatewaySenderQueue();
 
     // Add an event to the BucketRegionQueue and verify BucketRegionQueue state
-    GatewaySenderEventImpl gsei = this.bucketRegionQueueHelper.addEvent(KEY);
+    GatewaySenderEventImpl event = this.bucketRegionQueueHelper.addEvent(KEY);
     assertEquals(1, this.bucketRegionQueue.size());
 
     // Add a mock GatewaySenderEventImpl to the temp queue
-    BlockingQueue<GatewaySenderEventImpl> tempQueue = createTempQueueAndAddEvent(pgsep, gsei);
+    BlockingQueue<GatewaySenderEventImpl> tempQueue = createTempQueueAndAddEvent(processor, event);
     assertEquals(1, tempQueue.size());
 
     // Create and process a ParallelQueueRemovalMessage (causes the value of the entry to be set to
@@ -230,9 +254,9 @@ public class ParallelQueueRemovalMessageJUnitTest {
   }
 
   private void createAndProcessParallelQueueRemovalMessage() {
-    ParallelQueueRemovalMessage pqrm =
+    ParallelQueueRemovalMessage message =
         new ParallelQueueRemovalMessage(createRegionToDispatchedKeysMap());
-    pqrm.process(null);
+    message.process(null);
   }
 
   private HashMap<String, Map<Integer, List<Long>>> createRegionToDispatchedKeysMap() {
@@ -246,23 +270,23 @@ public class ParallelQueueRemovalMessageJUnitTest {
   }
 
   private ParallelGatewaySenderEventProcessor createConcurrentParallelGatewaySenderQueue() {
-    ParallelGatewaySenderEventProcessor pgsep = new ParallelGatewaySenderEventProcessor(sender);
-    ConcurrentParallelGatewaySenderQueue cpgsq = new ConcurrentParallelGatewaySenderQueue(sender,
-        new ParallelGatewaySenderEventProcessor[] {pgsep});
+    ParallelGatewaySenderEventProcessor processor = new ParallelGatewaySenderEventProcessor(sender);
+    ConcurrentParallelGatewaySenderQueue queue = new ConcurrentParallelGatewaySenderQueue(sender,
+        new ParallelGatewaySenderEventProcessor[] {processor});
     Set<RegionQueue> queues = new HashSet<>();
-    queues.add(cpgsq);
+    queues.add(queue);
     when(this.sender.getQueues()).thenReturn(queues);
-    return pgsep;
+    return processor;
   }
 
   private BlockingQueue<GatewaySenderEventImpl> createTempQueueAndAddEvent(
-      ParallelGatewaySenderEventProcessor pgsep, GatewaySenderEventImpl gsei) {
-    ParallelGatewaySenderQueue pgsq = (ParallelGatewaySenderQueue) pgsep.getQueue();
+      ParallelGatewaySenderEventProcessor processor, GatewaySenderEventImpl event) {
+    ParallelGatewaySenderQueue queue = (ParallelGatewaySenderQueue) processor.getQueue();
     Map<Integer, BlockingQueue<GatewaySenderEventImpl>> tempQueueMap =
-        pgsq.getBucketToTempQueueMap();
-    BlockingQueue<GatewaySenderEventImpl> tempQueue = new LinkedBlockingQueue();
-    when(gsei.getShadowKey()).thenReturn(KEY);
-    tempQueue.add(gsei);
+        queue.getBucketToTempQueueMap();
+    BlockingQueue<GatewaySenderEventImpl> tempQueue = new LinkedBlockingQueue<>();
+    when(event.getShadowKey()).thenReturn(KEY);
+    tempQueue.add(event);
     tempQueueMap.put(BUCKET_ID, tempQueue);
     return tempQueue;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-cq/src/test/java/org/apache/geode/cache/query/dunit/QueryMonitorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/cache/query/dunit/QueryMonitorDUnitTest.java b/geode-cq/src/test/java/org/apache/geode/cache/query/dunit/QueryMonitorDUnitTest.java
index f298fae..9332388 100644
--- a/geode-cq/src/test/java/org/apache/geode/cache/query/dunit/QueryMonitorDUnitTest.java
+++ b/geode-cq/src/test/java/org/apache/geode/cache/query/dunit/QueryMonitorDUnitTest.java
@@ -220,12 +220,12 @@ public class QueryMonitorDUnitTest extends JUnit4CacheTestCase {
       Assert.fail("While starting CacheServer", ex);
     }
     Cache cache = getCache();
-    GemFireCacheImpl.getInstance().TEST_MAX_QUERY_EXECUTION_TIME = queryMonitorTime;
+    GemFireCacheImpl.getInstance().testMaxQueryExecutionTime = queryMonitorTime;
     cache.getLogger().fine("#### RUNNING TEST : " + testName);
     DefaultQuery.testHook = new QueryTimeoutHook(queryMonitorTime);
-    // ((GemFireCache)cache).TEST_MAX_QUERY_EXECUTION_TIME = queryMonitorTime;
+    // ((GemFireCache)cache).testMaxQueryExecutionTime = queryMonitorTime;
     System.out.println("MAX_QUERY_EXECUTION_TIME is set to: "
-        + ((GemFireCacheImpl) cache).TEST_MAX_QUERY_EXECUTION_TIME);
+        + ((GemFireCacheImpl) cache).testMaxQueryExecutionTime);
     return port;
   }
 
@@ -236,10 +236,10 @@ public class QueryMonitorDUnitTest extends JUnit4CacheTestCase {
         // Reset the test flag.
         Cache cache = getCache();
         DefaultQuery.testHook = null;
-        GemFireCacheImpl.getInstance().TEST_MAX_QUERY_EXECUTION_TIME = -1;
+        GemFireCacheImpl.getInstance().testMaxQueryExecutionTime = -1;
         stopBridgeServer(getCache());
         System.out.println("MAX_QUERY_EXECUTION_TIME is set to: "
-            + ((GemFireCacheImpl) cache).TEST_MAX_QUERY_EXECUTION_TIME);
+            + ((GemFireCacheImpl) cache).testMaxQueryExecutionTime);
       }
     };
     server.invoke(stopServer);
@@ -333,7 +333,7 @@ public class QueryMonitorDUnitTest extends JUnit4CacheTestCase {
   private void executeQueriesFromClient(int timeout) {
     try {
       ClientCache anyInstance = ClientCacheFactory.getAnyInstance();
-      ((GemFireCacheImpl) anyInstance).TEST_MAX_QUERY_EXECUTION_TIME = timeout;
+      ((GemFireCacheImpl) anyInstance).testMaxQueryExecutionTime = timeout;
       Pool pool = PoolManager.find(poolName);
       QueryService queryService = pool.getQueryService();
       executeQueriesAgainstQueryService(queryService);
@@ -882,7 +882,7 @@ public class QueryMonitorDUnitTest extends JUnit4CacheTestCase {
   protected CqQueryDUnitTest cqDUnitTest = new CqQueryDUnitTest();
 
   /**
-   * The following CQ test is added to make sure TEST_MAX_QUERY_EXECUTION_TIME is reset and is not
+   * The following CQ test is added to make sure testMaxQueryExecutionTime is reset and is not
    * affecting other query related tests.
    * 
    * @throws Exception
@@ -917,7 +917,7 @@ public class QueryMonitorDUnitTest extends JUnit4CacheTestCase {
       public void run2() throws CacheException {
         Cache cache = getCache();
         System.out.println("TEST CQ MAX_QUERY_EXECUTION_TIME is set to: "
-            + ((GemFireCacheImpl) cache).TEST_MAX_QUERY_EXECUTION_TIME);
+            + ((GemFireCacheImpl) cache).testMaxQueryExecutionTime);
 
         Region region1 = getRootRegion().getSubregion(cqDUnitTest.regions[0]);
         for (int i = 1; i <= 5; i++) {
@@ -942,7 +942,7 @@ public class QueryMonitorDUnitTest extends JUnit4CacheTestCase {
       public void run2() throws CacheException {
         Cache cache = getCache();
         System.out.println("TEST CQ MAX_QUERY_EXECUTION_TIME is set to: "
-            + ((GemFireCacheImpl) cache).TEST_MAX_QUERY_EXECUTION_TIME);
+            + ((GemFireCacheImpl) cache).testMaxQueryExecutionTime);
 
         Region region1 = getRootRegion().getSubregion(cqDUnitTest.regions[0]);
         for (int i = 1; i <= 5; i++) {
@@ -1127,7 +1127,7 @@ public class QueryMonitorDUnitTest extends JUnit4CacheTestCase {
               }
               break;
             }
-            // ((GemFireCache)cache).TEST_MAX_QUERY_EXECUTION_TIME = queryMonitorTime;
+            // ((GemFireCache)cache).testMaxQueryExecutionTime = queryMonitorTime;
           }
         };
     vm.invoke(validateThreadCnt);

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java
index 0449a45..3190cd7 100644
--- a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java
@@ -38,7 +38,7 @@ import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.FunctionException;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.execute.util.FindRestEnabledServersFunction;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.rest.internal.web.controllers.support.RestServersResultCollector;
 import org.apache.geode.rest.internal.web.exception.GemfireRestException;
@@ -193,7 +193,7 @@ public abstract class CommonCrudController extends AbstractBaseController {
 
     try {
       final ResultCollector<?, ?> results = function.withCollector(new RestServersResultCollector())
-          .execute(GemFireCacheImpl.FIND_REST_ENABLED_SERVERS_FUNCTION_ID);
+          .execute(FindRestEnabledServersFunction.FIND_REST_ENABLED_SERVERS_FUNCTION_ID);
       Object functionResult = results.getResult();
 
       if (functionResult instanceof List<?>) {


[13/13] geode git commit: Safe refactorings

Posted by kl...@apache.org.
Safe refactorings


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

Branch: refs/heads/feature/GEODE-2632-6-1
Commit: 915a0c53c28bca34f445834768a88b9db987148c
Parents: 9f8ba8d
Author: Kirk Lund <kl...@apache.org>
Authored: Mon Apr 24 10:38:37 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Mon Apr 24 13:19:16 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/geode/CancelCriterion.java  |   23 +-
 .../main/java/org/apache/geode/cache/Cache.java |  132 +-
 .../geode/cache/client/internal/ProxyCache.java |   18 +-
 .../cache/query/internal/QueryMonitor.java      |    6 +-
 .../apache/geode/distributed/internal/DM.java   |    5 +-
 .../internal/DistributionManager.java           |    2 +-
 .../internal/InternalDistributedSystem.java     |    7 +-
 .../internal/LonerDistributionManager.java      |    5 +
 .../geode/internal/cache/DistTXState.java       |    6 +-
 .../geode/internal/cache/GemFireCacheImpl.java  | 2249 ++++++++----------
 .../geode/internal/cache/InternalCache.java     |    4 +-
 .../geode/internal/cache/LocalRegion.java       |    4 +-
 .../util/FindRestEnabledServersFunction.java    |    8 +-
 .../persistence/PersistenceAdvisorImpl.java     |    8 +-
 .../internal/cache/xmlcache/CacheCreation.java  |    2 +-
 .../internal/beans/MemberMBeanBridge.java       |   13 +-
 .../test/java/org/apache/geode/TXJUnitTest.java |  844 ++++---
 .../java/org/apache/geode/TXWriterTestCase.java |   67 +-
 ...esourceManagerWithQueryMonitorDUnitTest.java |   11 +-
 .../geode/disttx/DistTXDebugDUnitTest.java      |   98 +-
 .../apache/geode/disttx/DistTXJUnitTest.java    |   37 +-
 .../disttx/DistTXPersistentDebugDUnitTest.java  |   15 +-
 .../geode/disttx/DistTXWriterJUnitTest.java     |   37 +-
 .../geode/disttx/DistTXWriterOOMEJUnitTest.java |   39 +-
 .../disttx/DistributedTransactionDUnitTest.java |    9 +-
 .../apache/geode/disttx/PRDistTXJUnitTest.java  |   25 +-
 .../geode/internal/cache/PRTXJUnitTest.java     |  106 +-
 .../ParallelQueueRemovalMessageJUnitTest.java   |  104 +-
 .../query/dunit/QueryMonitorDUnitTest.java      |   20 +-
 .../web/controllers/CommonCrudController.java   |    4 +-
 30 files changed, 1805 insertions(+), 2103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/CancelCriterion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/CancelCriterion.java b/geode-core/src/main/java/org/apache/geode/CancelCriterion.java
index e4f9a41..fec3827 100644
--- a/geode-core/src/main/java/org/apache/geode/CancelCriterion.java
+++ b/geode-core/src/main/java/org/apache/geode/CancelCriterion.java
@@ -22,20 +22,20 @@ package org.apache.geode;
  * 
  * Code inside the service can check to see if the service is cancelled by calling
  * {@link #checkCancelInProgress(Throwable)}. Generally the pattern is to check before performing an
- * operation, check if the service is canceled before propgrating an exception futher up the stack,
- * and check for cancelation inside a long loop. Eg.
+ * operation, check if the service is canceled before propagating an exception further up the stack,
+ * and check for cancellation inside a long loop. Eg.
  * 
- * <code>
- * while(true) {
+ * <pre>
+ * while (true) {
  *   c.checkCancelInProgress(null);
  *   try {
- *      dispatchEvents();
- *   } catch(IOException e) {
+ *     dispatchEvents();
+ *   } catch (IOException e) {
  *     c.checkCancelInProgress(e);
  *     throw e;
  *   }
  * }
- * </code>
+ * </pre>
  * 
  * @see CancelException
  * @since GemFire 5.1
@@ -51,10 +51,6 @@ public abstract class CancelCriterion {
    *         exception indicating the service is shut down.
    */
   public abstract String cancelInProgress();
-  // import org.apache.geode.distributed.internal.DistributionManager;
-  // * <p>
-  // * In particular, a {@link DistributionManager} returns a non-null result if
-  // * message distribution has been terminated.
 
   /**
    * Use this utility function in your implementation of cancelInProgress() and cancelled() to
@@ -95,11 +91,11 @@ public abstract class CancelCriterion {
    * This method should wrap the exception in a service specific CancelationException (eg
    * CacheClosedException). or return null if the service is not being canceled.
    * 
-   * @param e an underlying exception, if any
+   * @param throwable an underlying exception, if any
    * @return RuntimeException to be thrown by checkCancelInProgress(), null if the receiver has not
    *         been cancelled.
    */
-  abstract public RuntimeException generateCancelledException(Throwable e);
+  public abstract RuntimeException generateCancelledException(Throwable throwable);
 
   /**
    * Checks to see if a cancellation is in progress. This is equivalent to the expression
@@ -111,5 +107,4 @@ public abstract class CancelCriterion {
     return cancelInProgress() != null;
   }
 
-
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/cache/Cache.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/Cache.java b/geode-core/src/main/java/org/apache/geode/cache/Cache.java
index bc4aa19..66a3cd8 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/Cache.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/Cache.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.cache;
 
 import java.util.List;
@@ -34,7 +33,6 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.i18n.LogWriterI18n;
 
-
 /**
  * Caches are obtained from the {@link CacheFactory#create()} method. See {@link CacheFactory} for
  * common usage patterns for creating the cache instance.
@@ -43,17 +41,15 @@ import org.apache.geode.i18n.LogWriterI18n;
  * where to find other caches on the network and how to communicate with them. The system can also
  * specify a <a href="../distribution/DistributedSystem.html#cache-xml-file">"cache-xml-file"</a>
  * property which will cause this cache to be initialized with the contents of that file. The
- * contents must comply with the <code>"doc-files/cache8_0.dtd"</code> file and the top level
- * element must be a <code>cache</code> element.
+ * contents must comply with the {@code "doc-files/cache8_0.dtd"} file and the top level element
+ * must be a {@code cache} element.
  * <p>
  * When a cache will no longer be used it should be {@link #close() closed}. Once it
  * {@link #isClosed is closed} any attempt to use it or any {@link Region} obtained from it will
  * cause a {@link CacheClosedException} to be thrown.
- *
  * <p>
  * A cache can have multiple root regions, each with a different name.
  *
- *
  * @since GemFire 2.0
  */
 @SuppressWarnings("deprecation")
@@ -63,13 +59,13 @@ public interface Cache extends GemFireCache {
    * region in the cache. After this cache is closed, any further method call on this cache or any
    * region object will throw {@link CacheClosedException}, unless otherwise noted.
    * 
-   * @param keepalive whether the server should keep the durable client's queues alive for the
+   * @param keepAlive whether the server should keep the durable client's queues alive for the
    *        timeout period
    * @throws CacheClosedException if the cache is already closed.
    * @deprecated as of 6.5 use {@link ClientCache#close(boolean)} instead.
    */
   @Deprecated
-  public void close(boolean keepalive);
+  void close(boolean keepAlive);
 
   /**
    * Creates a VM region using the specified RegionAttributes.
@@ -88,7 +84,7 @@ public interface Cache extends GemFireCache {
    * @deprecated as of GemFire 5.0, use {@link #createRegion} instead.
    */
   @Deprecated
-  public <K, V> Region<K, V> createVMRegion(String name, RegionAttributes<K, V> aRegionAttributes)
+  <K, V> Region<K, V> createVMRegion(String name, RegionAttributes<K, V> aRegionAttributes)
       throws RegionExistsException, TimeoutException;
 
   /**
@@ -109,7 +105,7 @@ public interface Cache extends GemFireCache {
    * @deprecated as of 6.5 use {@link #createRegionFactory(RegionAttributes)} instead
    */
   @Deprecated
-  public <K, V> Region<K, V> createRegion(String name, RegionAttributes<K, V> aRegionAttributes)
+  <K, V> Region<K, V> createRegion(String name, RegionAttributes<K, V> aRegionAttributes)
       throws RegionExistsException, TimeoutException;
 
   /**
@@ -119,7 +115,7 @@ public interface Cache extends GemFireCache {
    * @see #createRegionFactory(RegionShortcut)
    * @since GemFire 6.5
    */
-  public <K, V> RegionFactory<K, V> createRegionFactory();
+  <K, V> RegionFactory<K, V> createRegionFactory();
 
   /**
    * Creates a {@link RegionFactory} for the most commonly used {@link Region} types defined by
@@ -127,7 +123,7 @@ public interface Cache extends GemFireCache {
    * 
    * @since GemFire 6.5
    */
-  public <K, V> RegionFactory<K, V> createRegionFactory(RegionShortcut atts);
+  <K, V> RegionFactory<K, V> createRegionFactory(RegionShortcut shortcut);
 
   /**
    * Creates a {@link RegionFactory} for creating a {@link Region} from {@link RegionAttributes}
@@ -137,7 +133,7 @@ public interface Cache extends GemFireCache {
    * @see #setRegionAttributes(String, RegionAttributes)
    * @since GemFire 6.5
    */
-  public <K, V> RegionFactory<K, V> createRegionFactory(String regionAttributesId);
+  <K, V> RegionFactory<K, V> createRegionFactory(String regionAttributesId);
 
   /**
    * Creates a {@link RegionFactory} for creating a {@link Region} from the given regionAttributes
@@ -146,43 +142,43 @@ public interface Cache extends GemFireCache {
    * @see #createRegionFactory(RegionShortcut)
    * @since GemFire 6.5
    */
-  public <K, V> RegionFactory<K, V> createRegionFactory(RegionAttributes<K, V> regionAttributes);
+  <K, V> RegionFactory<K, V> createRegionFactory(RegionAttributes<K, V> regionAttributes);
 
   /**
    * Internal GemStone method for accessing the internationalized logging object for GemFire, use
-   * {@link #getLogger()} instead. This method does not throw <code>CacheClosedException</code> if
-   * the cache is closed.
+   * {@link #getLogger()} instead. This method does not throw {@code CacheClosedException} if the
+   * cache is closed.
    * 
    * @return the logging object
    * @deprecated as of 6.5 use getLogger().convertToLogWriterI18n() instead
    */
   @Deprecated
-  public LogWriterI18n getLoggerI18n();
+  LogWriterI18n getLoggerI18n();
 
   /**
    * Internal GemStone method for accessing the internationalized logging object for GemFire, use
-   * {@link #getSecurityLogger()} instead. This method does not throw
-   * <code>CacheClosedException</code> if the cache is closed.
+   * {@link #getSecurityLogger()} instead. This method does not throw {@code CacheClosedException}
+   * if the cache is closed.
    * 
    * @return the security logging object
    * @deprecated as of 6.5 use getSecurityLogger().convertToLogWriterI18n() instead
    */
   @Deprecated
-  public LogWriterI18n getSecurityLoggerI18n();
+  LogWriterI18n getSecurityLoggerI18n();
 
   /**
    * Gets the number of seconds a cache operation will wait to obtain a distributed lock lease. This
-   * method does not throw <code>CacheClosedException</code> if the cache is closed.
+   * method does not throw {@code CacheClosedException} if the cache is closed.
    */
-  public int getLockTimeout();
+  int getLockTimeout();
 
   /**
    * Sets the number of seconds a cache operation may wait to obtain a distributed lock lease before
    * timing out.
    *
-   * @throws IllegalArgumentException if <code>seconds</code> is less than zero
+   * @throws IllegalArgumentException if {@code seconds} is less than zero
    */
-  public void setLockTimeout(int seconds);
+  void setLockTimeout(int seconds);
 
   /**
    * Gets the frequency (in seconds) at which a message will be sent by the primary cache-server to
@@ -191,45 +187,45 @@ public interface Cache extends GemFireCache {
    * 
    * @return The time interval in seconds
    */
-  public int getMessageSyncInterval();
+  int getMessageSyncInterval();
 
   /**
    * Sets the frequency (in seconds) at which a message will be sent by the primary cache-server
    * node to all the secondary cache-server nodes to remove the events which have already been
    * dispatched from the queue.
    * 
-   * @param seconds - the time interval in seconds
-   * @throws IllegalArgumentException if <code>seconds</code> is less than zero
+   * @param seconds the time interval in seconds
+   * @throws IllegalArgumentException if {@code seconds} is less than zero
    */
-  public void setMessageSyncInterval(int seconds);
+  void setMessageSyncInterval(int seconds);
 
   /**
    * Gets the length, in seconds, of distributed lock leases obtained by this cache. This method
-   * does not throw <code>CacheClosedException</code> if the cache is closed.
+   * does not throw {@code CacheClosedException} if the cache is closed.
    */
-  public int getLockLease();
+  int getLockLease();
 
   /**
    * Sets the length, in seconds, of distributed lock leases obtained by this cache.
    *
-   * @throws IllegalArgumentException if <code>seconds</code> is less than zero.
+   * @throws IllegalArgumentException if {@code seconds} is less than zero.
    */
-  public void setLockLease(int seconds);
+  void setLockLease(int seconds);
 
   /**
    * Gets the number of seconds a cache {@link org.apache.geode.cache.Region#get(Object) get}
    * operation can spend searching for a value before it times out. The search includes any time
    * spent loading the object. When the search times out it causes the get to fail by throwing an
-   * exception. This method does not throw <code>CacheClosedException</code> if the cache is closed.
+   * exception. This method does not throw {@code CacheClosedException} if the cache is closed.
    */
-  public int getSearchTimeout();
+  int getSearchTimeout();
 
   /**
    * Sets the number of seconds a cache get operation can spend searching for a value.
    *
-   * @throws IllegalArgumentException if <code>seconds</code> is less than zero
+   * @throws IllegalArgumentException if {@code seconds} is less than zero
    */
-  public void setSearchTimeout(int seconds);
+  void setSearchTimeout(int seconds);
 
   /**
    * Creates a new cache server, with the default configuration, that will allow clients to access
@@ -242,17 +238,17 @@ public interface Cache extends GemFireCache {
    *
    * @since GemFire 5.7
    */
-  public CacheServer addCacheServer();
+  CacheServer addCacheServer();
 
   /**
-   * Returns a collection of all of the <code>CacheServer</code>s that can serve the contents of
-   * this <code>Cache</code> to clients.
+   * Returns a collection of all of the {@code CacheServer}s that can serve the contents of this
+   * {@code Cache} to clients.
    *
    * @see #addCacheServer
    *
    * @since GemFire 5.7
    */
-  public List<CacheServer> getCacheServers();
+  List<CacheServer> getCacheServers();
 
   /**
    * Adds a gateway event conflict resolution resolver. This is invoked if an event is processed
@@ -262,27 +258,27 @@ public interface Cache extends GemFireCache {
    * event's distributed system ID is larger than that of the last event to modify the affected
    * entry.
    * 
-   * @param resolver
+   * @param resolver gateway event conflict resolution resolver
    * @since GemFire 7.0
    */
-  public void setGatewayConflictResolver(GatewayConflictResolver resolver);
+  void setGatewayConflictResolver(GatewayConflictResolver resolver);
 
   /**
    * Returns the current gateway event conflict resolver
    * 
    * @since GemFire 7.0
    */
-  public GatewayConflictResolver getGatewayConflictResolver();
+  GatewayConflictResolver getGatewayConflictResolver();
 
   /**
-   * Sets whether or not this <code>Cache</code> resides in a long-running "cache server" VM. A
-   * cache server may be an application VM or may be a stand-along VM launched using
+   * Sets whether or not this {@code Cache} resides in a long-running "cache server" VM. A cache
+   * server may be an application VM or may be a stand-along VM launched using
    * {@linkplain org.apache.geode.admin.AdminDistributedSystem#addCacheServer administration API} or
-   * the <code>cacheserver</code> command line utility.
+   * the {@code cacheserver} command line utility.
    *
    * @since GemFire 4.0
    */
-  public void setIsServer(boolean isServer);
+  void setIsServer(boolean isServer);
 
   /**
    * Returns whether or not this cache resides in a "cache server" VM.
@@ -291,7 +287,7 @@ public interface Cache extends GemFireCache {
    *
    * @since GemFire 4.0
    */
-  public boolean isServer();
+  boolean isServer();
 
   /**
    * Notifies the server that this client is ready to receive updates. This method is used by
@@ -309,7 +305,7 @@ public interface Cache extends GemFireCache {
    * @deprecated as of 6.5 use {@link ClientCache#readyForEvents} instead.
    */
   @Deprecated
-  public void readyForEvents();
+  void readyForEvents();
 
   /**
    * Creates {@link GatewaySenderFactory} for creating a SerialGatewaySender
@@ -317,7 +313,8 @@ public interface Cache extends GemFireCache {
    * @return SerialGatewaySenderFactory
    * @since GemFire 7.0
    */
-  public GatewaySenderFactory createGatewaySenderFactory();
+  @Override
+  GatewaySenderFactory createGatewaySenderFactory();
 
   /**
    * Creates {@link AsyncEventQueueFactory} for creating a AsyncEventQueue
@@ -325,7 +322,7 @@ public interface Cache extends GemFireCache {
    * @return AsyncEventQueueFactory
    * @since GemFire 7.0
    */
-  public AsyncEventQueueFactory createAsyncEventQueueFactory();
+  AsyncEventQueueFactory createAsyncEventQueueFactory();
 
   /**
    * Creates {@link GatewayReceiverFactory} for creating a GatewayReceiver
@@ -333,7 +330,7 @@ public interface Cache extends GemFireCache {
    * @return GatewayReceiverFactory
    * @since GemFire 7.0
    */
-  public GatewayReceiverFactory createGatewayReceiverFactory();
+  GatewayReceiverFactory createGatewayReceiverFactory();
 
   /**
    * Returns all {@link GatewaySender}s for this Cache.
@@ -341,7 +338,7 @@ public interface Cache extends GemFireCache {
    * @return Set of GatewaySenders
    * @since GemFire 7.0
    */
-  public Set<GatewaySender> getGatewaySenders();
+  Set<GatewaySender> getGatewaySenders();
 
   /**
    * Returns the {@link GatewaySender} with the given id added to this Cache.
@@ -349,7 +346,7 @@ public interface Cache extends GemFireCache {
    * @return GatewaySender with id
    * @since GemFire 7.0
    */
-  public GatewaySender getGatewaySender(String id);
+  GatewaySender getGatewaySender(String id);
 
   /**
    * Returns all {@link GatewayReceiver}s for this Cache
@@ -357,7 +354,7 @@ public interface Cache extends GemFireCache {
    * @return Set of GatewaySenders
    * @since GemFire 7.0
    */
-  public Set<GatewayReceiver> getGatewayReceivers();
+  Set<GatewayReceiver> getGatewayReceivers();
 
   /**
    * Returns all {@link AsyncEventQueue}s for this Cache
@@ -365,7 +362,7 @@ public interface Cache extends GemFireCache {
    * @return Set of AsyncEventQueue
    * @since GemFire 7.0
    */
-  public Set<AsyncEventQueue> getAsyncEventQueues();
+  Set<AsyncEventQueue> getAsyncEventQueues();
 
   /**
    * Returns the {@link AsyncEventQueue} with the given id added to this Cache.
@@ -373,37 +370,37 @@ public interface Cache extends GemFireCache {
    * @return AsyncEventQueue with id
    * @since GemFire 7.0
    */
-  public AsyncEventQueue getAsyncEventQueue(String id);
+  AsyncEventQueue getAsyncEventQueue(String id);
 
   /**
    * Returns a set of the other non-administrative members in the distributed system.
    * 
    * @since GemFire 6.6
    */
-  public Set<DistributedMember> getMembers();
+  Set<DistributedMember> getMembers();
 
   /**
    * Returns a set of the administrative members in the distributed system.
    * 
    * @since GemFire 6.6
    */
-  public Set<DistributedMember> getAdminMembers();
+  Set<DistributedMember> getAdminMembers();
 
   /**
    * Returns a set of the members in the distributed system that have the given region. For regions
    * with local scope an empty set is returned.
    * 
-   * @param r a Region in the cache
+   * @param region a Region in the cache
    * @since GemFire 6.6
    */
-  public Set<DistributedMember> getMembers(Region r);
+  Set<DistributedMember> getMembers(Region region);
 
   /**
    * Obtains the snapshot service to allow the cache data to be imported or exported.
    * 
    * @return the snapshot service
    */
-  public CacheSnapshotService getSnapshotService();
+  CacheSnapshotService getSnapshotService();
 
   /**
    * Test to see whether the Cache is in the process of reconnecting and recreating a new cache
@@ -415,30 +412,29 @@ public interface Cache extends GemFireCache {
    * 
    * @return true if the Cache is attempting to reconnect or has finished reconnecting
    */
-  public boolean isReconnecting();
+  boolean isReconnecting();
 
   /**
    * Wait for the Cache to finish reconnecting to the distributed system and recreate a new Cache.
    * 
    * @see #getReconnectedCache
    * @param time amount of time to wait, or -1 to wait forever
-   * @param units
+   * @param units time unit
    * @return true if the cache was reconnected
    * @throws InterruptedException if the thread is interrupted while waiting
    */
-  public boolean waitUntilReconnected(long time, TimeUnit units) throws InterruptedException;
+  boolean waitUntilReconnected(long time, TimeUnit units) throws InterruptedException;
 
   /**
    * Force the Cache to stop reconnecting. If the Cache is currently connected this will disconnect
    * and close it.
    * 
    */
-  public void stopReconnecting();
+  void stopReconnecting();
 
   /**
    * Returns the new Cache if there was an auto-reconnect and the cache was recreated.
    */
-  public Cache getReconnectedCache();
+  Cache getReconnectedCache();
 
 }
-

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
index 76306f5..f4a8d5b 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
@@ -221,22 +221,16 @@ public class ProxyCache implements RegionService {
     return this.stopper;
   }
 
-  /*
-   * (non-Javadoc)
-   * 
-   * @see org.apache.geode.cache.RegionService#rootRegions()
-   */
+  @Override
   public Set<Region<?, ?>> rootRegions() {
     preOp();
-    Set<Region<?, ?>> rRegions = new HashSet<Region<?, ?>>();
-    Iterator<LocalRegion> it = this.cache.rootRegions().iterator();
-    while (it.hasNext()) {
-      LocalRegion lr = it.next();
-      if (!lr.getAttributes().getDataPolicy().withStorage()) {
-        rRegions.add(new ProxyRegion(this, lr));
+    Set<Region<?, ?>> rootRegions = new HashSet<>();
+    for (Region<?, ?> region : this.cache.rootRegions()) {
+      if (!region.getAttributes().getDataPolicy().withStorage()) {
+        rootRegions.add(new ProxyRegion(this, region));
       }
     }
-    return Collections.unmodifiableSet(rRegions);
+    return Collections.unmodifiableSet(rootRegions);
   }
 
   public PdxInstanceFactory createPdxInstanceFactory(String className) {

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/cache/query/internal/QueryMonitor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/QueryMonitor.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/QueryMonitor.java
index d6acfbf..569fbb0 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/QueryMonitor.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/QueryMonitor.java
@@ -103,7 +103,7 @@ public class QueryMonitor implements Runnable {
 
     /** For dunit test purpose */
     if (GemFireCacheImpl.getInstance() != null
-        && GemFireCacheImpl.getInstance().TEST_MAX_QUERY_EXECUTION_TIME > 0) {
+        && GemFireCacheImpl.getInstance().testMaxQueryExecutionTime > 0) {
       if (this.queryMonitorTasks == null) {
         this.queryMonitorTasks = new ConcurrentHashMap();
       }
@@ -127,8 +127,8 @@ public class QueryMonitor implements Runnable {
 
       // START - DUnit Test purpose.
       if (GemFireCacheImpl.getInstance() != null
-          && GemFireCacheImpl.getInstance().TEST_MAX_QUERY_EXECUTION_TIME > 0) {
-        long maxTimeSet = GemFireCacheImpl.getInstance().TEST_MAX_QUERY_EXECUTION_TIME;
+          && GemFireCacheImpl.getInstance().testMaxQueryExecutionTime > 0) {
+        long maxTimeSet = GemFireCacheImpl.getInstance().testMaxQueryExecutionTime;
         QueryThreadTask queryTask = (QueryThreadTask) queryThreads.peek();
 
         long currentTime = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/distributed/internal/DM.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DM.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DM.java
index 328a4f8..afc8125 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DM.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DM.java
@@ -33,9 +33,6 @@ import org.apache.geode.internal.Version;
 
 /**
  * This interface defines the services provided by any class that is a distribution manager.
- *
- *
- *
  */
 public interface DM extends ReplySender {
 
@@ -463,4 +460,6 @@ public interface DM extends ReplySender {
    * forceUDPMessagingForCurrentThread.
    */
   public void releaseUDPMessagingForCurrentThread();
+
+  int getDMType();
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
index 2ae86e6..6920311 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
@@ -1756,7 +1756,7 @@ public class DistributionManager implements DM {
   }
 
   /**
-   * Add a membership listener and return other DistribtionManagerIds as an atomic operation
+   * Add a membership listener and return other DistributionManagerIds as an atomic operation
    */
   public Set addMembershipListenerAndGetDistributionManagerIds(MembershipListener l) {
     // switched sync order to fix bug 30360

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index 987e491..86bc7a4 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -117,6 +117,11 @@ import org.apache.geode.security.GemFireSecurityException;
 public class InternalDistributedSystem extends DistributedSystem
     implements OsStatisticsFactory, StatisticsManager {
 
+  /**
+   * True if the user is allowed lock when memory resources appear to be overcommitted.
+   */
+  private static final boolean ALLOW_MEMORY_LOCK_WHEN_OVERCOMMITTED =
+      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "Cache.ALLOW_MEMORY_OVERCOMMIT");
   private static final Logger logger = LogService.getLogger();
 
   public static final String DISABLE_MANAGEMENT_PROPERTY =
@@ -654,7 +659,7 @@ public class InternalDistributedSystem extends DistributedSystem
         long avail = LinuxProcFsStatistics.getAvailableMemory(logger);
         long size = offHeapMemorySize + Runtime.getRuntime().totalMemory();
         if (avail < size) {
-          if (GemFireCacheImpl.ALLOW_MEMORY_LOCK_WHEN_OVERCOMMITTED) {
+          if (ALLOW_MEMORY_LOCK_WHEN_OVERCOMMITTED) {
             logger.warn(LocalizedMessage.create(
                 LocalizedStrings.InternalDistributedSystem_MEMORY_OVERCOMMIT_WARN, size - avail));
           } else {

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
index af4e674..e9068e6 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
@@ -1357,6 +1357,11 @@ public class LonerDistributionManager implements DM {
   }
 
   @Override
+  public int getDMType() {
+    return 0;
+  }
+
+  @Override
   public boolean isSharedConfigurationServiceEnabledForDS() {
     // return false for loner
     return false;

http://git-wip-us.apache.org/repos/asf/geode/blob/915a0c53/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXState.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXState.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXState.java
index 6df2623..226ffa6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXState.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistTXState.java
@@ -51,6 +51,8 @@ import org.apache.geode.internal.offheap.annotations.Released;
  */
 public class DistTXState extends TXState {
 
+  public static Runnable internalBeforeApplyChanges;
+  public static Runnable internalBeforeNonTXBasicPut;
   private boolean updatingTxStateDuringPreCommit = false;
 
   public DistTXState(TXStateProxy proxy, boolean onBehalfOfRemoteStub) {
@@ -263,8 +265,8 @@ public class DistTXState extends TXState {
       try {
         attachFilterProfileInformation(entries);
 
-        if (GemFireCacheImpl.internalBeforeApplyChanges != null) {
-          GemFireCacheImpl.internalBeforeApplyChanges.run();
+        if (internalBeforeApplyChanges != null) {
+          internalBeforeApplyChanges.run();
         }
 
         // apply changes to the cache


[06/13] geode git commit: GEODE-2632: refactor code to reduce GemFireCacheImpl dependencies

Posted by kl...@apache.org.
GEODE-2632: refactor code to reduce GemFireCacheImpl dependencies

* extract fetching GemFireCacheImpl to Provider interface/class
* use InternalCache instead of casting to Impl
* delete useless javadocs and comments
* reduce scope of constants, vars and methods


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

Branch: refs/heads/feature/GEODE-2632-6-1
Commit: 47d8c82036a9863c9fa7c9142c170c9f8552abb4
Parents: 60ec931
Author: Kirk Lund <kl...@apache.org>
Authored: Wed Apr 19 09:56:24 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Apr 21 14:47:50 2017 -0700

----------------------------------------------------------------------
 .../web/controllers/AbstractBaseController.java | 194 +++++++++----------
 .../web/controllers/BaseControllerAdvice.java   |  28 ++-
 .../web/controllers/CommonCrudController.java   |  29 +--
 .../controllers/FunctionAccessController.java   |  39 ++--
 .../web/controllers/PdxBasedCrudController.java |  28 ++-
 .../web/controllers/QueryAccessController.java  |  49 ++---
 .../web/controllers/support/CacheProvider.java  |  22 +++
 .../controllers/support/CacheProviderImpl.java  |  29 +++
 8 files changed, 219 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/47d8c820/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/AbstractBaseController.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/AbstractBaseController.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/AbstractBaseController.java
index 68080a3..d8eb572 100644
--- a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/AbstractBaseController.java
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/AbstractBaseController.java
@@ -12,15 +12,44 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.rest.internal.web.controllers;
 
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URLDecoder;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import javax.annotation.PostConstruct;
+
 import com.fasterxml.jackson.core.JsonParseException;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.JsonMappingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.logging.log4j.Logger;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+import org.json.JSONTokener;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.http.HttpHeaders;
+import org.springframework.http.HttpStatus;
+import org.springframework.http.ResponseEntity;
+import org.springframework.util.Assert;
+import org.springframework.util.ClassUtils;
+import org.springframework.util.CollectionUtils;
+import org.springframework.util.StringUtils;
+import org.springframework.web.servlet.support.ServletUriComponentsBuilder;
+
 import org.apache.geode.SerializationException;
-import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheLoaderException;
 import org.apache.geode.cache.CacheWriterException;
 import org.apache.geode.cache.LowMemoryException;
@@ -33,11 +62,12 @@ import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.LeaseExpiredException;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.pdx.JSONFormatter;
 import org.apache.geode.pdx.JSONFormatterException;
 import org.apache.geode.pdx.PdxInstance;
+import org.apache.geode.rest.internal.web.controllers.support.CacheProvider;
 import org.apache.geode.rest.internal.web.controllers.support.JSONTypes;
 import org.apache.geode.rest.internal.web.controllers.support.UpdateOp;
 import org.apache.geode.rest.internal.web.exception.DataTypeNotSupportedException;
@@ -51,79 +81,49 @@ import org.apache.geode.rest.internal.web.util.IdentifiableUtils;
 import org.apache.geode.rest.internal.web.util.JSONUtils;
 import org.apache.geode.rest.internal.web.util.NumberUtils;
 import org.apache.geode.rest.internal.web.util.ValidationUtils;
-import org.apache.logging.log4j.Logger;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.json.JSONTokener;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.http.HttpHeaders;
-import org.springframework.http.HttpStatus;
-import org.springframework.http.ResponseEntity;
-import org.springframework.util.Assert;
-import org.springframework.util.ClassUtils;
-import org.springframework.util.CollectionUtils;
-import org.springframework.util.StringUtils;
-import org.springframework.web.servlet.support.ServletUriComponentsBuilder;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URI;
-import java.net.URLDecoder;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-import javax.annotation.PostConstruct;
-
 
 /**
  * AbstractBaseController class contains common functionalities required for other controllers.
  * 
  * @since GemFire 8.0
  */
-
 @SuppressWarnings("unused")
 public abstract class AbstractBaseController {
 
-  protected static final String NEW_META_DATA_PROPERTY = "@new";
-  protected static final String OLD_META_DATA_PROPERTY = "@old";
-  protected static final String TYPE_META_DATA_PROPERTY = "@type";
-  protected static final String UTF_8 = "UTF-8";
-  protected static final String DEFAULT_ENCODING = UTF_8;
+  private static final String NEW_META_DATA_PROPERTY = "@new";
+  private static final String OLD_META_DATA_PROPERTY = "@old";
+  private static final String TYPE_META_DATA_PROPERTY = "@type";
+  private static final String UTF_8 = "UTF-8";
+  private static final String DEFAULT_ENCODING = UTF_8;
   private static final Logger logger = LogService.getLogger();
   private static final AtomicLong ID_SEQUENCE = new AtomicLong(0l);
 
-  // private Cache cache = GemFireCacheImpl.getExisting(null);
   @Autowired
-  protected RestSecurityService securityService;
+  private RestSecurityService securityService;
   @Autowired
   private ObjectMapper objectMapper;
+  @Autowired
+  private CacheProvider cacheProvider;
 
   @PostConstruct
   private void init() {
     JSONUtils.setObjectMapper(objectMapper);
   }
 
-  protected Cache getCache() {
-    Cache cache = GemFireCacheImpl.getExisting();
+  protected InternalCache getCache() {
+    InternalCache cache = cacheProvider.getInternalCache();
     Assert.state(cache != null, "The Gemfire Cache reference was not properly initialized");
     return cache;
   }
 
-  protected URI toUri(final String... pathSegments) {
+  URI toUri(final String... pathSegments) {
     return ServletUriComponentsBuilder.fromCurrentContextPath().path(getRestApiVersion())
         .pathSegment(pathSegments).build().toUri();
   }
 
   protected abstract String getRestApiVersion();
 
-  protected String validateQuery(String queryInUrl, String queryInBody) {
+  String validateQuery(String queryInUrl, String queryInBody) {
 
     if (!(StringUtils.hasText(queryInUrl) || StringUtils.hasText(queryInBody))) {
       throw new GemfireRestException("could not process null value specified in query String");
@@ -131,7 +131,7 @@ public abstract class AbstractBaseController {
     return (StringUtils.hasText(queryInUrl) ? decode(queryInUrl) : queryInBody);
   }
 
-  protected String decode(final String value) {
+  String decode(final String value) {
     if (value == null) {
       throw new GemfireRestException("could not process null value specified in query String");
     }
@@ -175,7 +175,7 @@ public abstract class AbstractBaseController {
   }
 
   @SuppressWarnings("unchecked")
-  protected <T> T casValue(String regionNamePath, String key, String jsonData) {
+  private <T> T casValue(String regionNamePath, String key, String jsonData) {
     JSONObject jsonObject;
     try {
       jsonObject = new JSONObject(jsonData);
@@ -189,7 +189,7 @@ public abstract class AbstractBaseController {
     }
   }
 
-  public ResponseEntity<String> processQueryResponse(Query query, Object args[], Object queryResult)
+  ResponseEntity<String> processQueryResponse(Query query, Object args[], Object queryResult)
       throws JSONException {
     if (queryResult instanceof Collection<?>) {
       Collection processedResults = new ArrayList(((Collection) queryResult).size());
@@ -207,7 +207,7 @@ public abstract class AbstractBaseController {
     }
   }
 
-  protected Collection<PdxInstance> convertJsonArrayIntoPdxCollection(final String jsonArray) {
+  Collection<PdxInstance> convertJsonArrayIntoPdxCollection(final String jsonArray) {
     JSONArray jsonArr = null;
     try {
       jsonArr = new JSONArray(jsonArray);
@@ -229,18 +229,7 @@ public abstract class AbstractBaseController {
     }
   }
 
-
-  /*
-   * protected PdxInstance convertJsonIntoPdxCollection(final String jsonArray) { JSONArray jsonArr
-   * = null;
-   * 
-   * PdxInstance pi = convert(jsonArray);
-   * System.out.println("Successfully converted into PdxInstance..!!"); return pi;
-   * 
-   * }
-   */
-
-  protected Object casValue(final String regionNamePath, final Object key, final Object oldValue,
+  private Object casValue(final String regionNamePath, final Object key, final Object oldValue,
       final Object newValue) {
     final Region<Object, Object> region = getRegion(regionNamePath);
     try {
@@ -281,7 +270,7 @@ public abstract class AbstractBaseController {
     }
   }
 
-  protected void replaceValue(final String regionNamePath, final Object key,
+  private void replaceValue(final String regionNamePath, final Object key,
       final PdxInstance value) {
     try {
       if (getRegion(regionNamePath).replace(key, value) == null) {
@@ -367,7 +356,7 @@ public abstract class AbstractBaseController {
     }
   }
 
-  protected void putValue(final String regionNamePath, final Object key, final Object value) {
+  private void putValue(final String regionNamePath, final Object key, final Object value) {
     try {
       getRegion(regionNamePath).put(key, value);
     } catch (NullPointerException npe) {
@@ -397,23 +386,23 @@ public abstract class AbstractBaseController {
     }
   }
 
-  protected void deleteQueryId(final String regionNamePath, final String key) {
+  private void deleteQueryId(final String regionNamePath, final String key) {
     getQueryStore(regionNamePath).remove(key);
   }
 
-  protected void deleteNamedQuery(final String regionNamePath, final String key) {
+  void deleteNamedQuery(final String regionNamePath, final String key) {
     // Check whether query ID exist in region or not
     checkForQueryIdExist(regionNamePath, key);
     deleteQueryId(regionNamePath, key);
   }
 
-  protected void checkForQueryIdExist(String region, String key) {
+  void checkForQueryIdExist(String region, String key) {
     if (!getQueryStore(region).containsKey(key)) {
       throw new ResourceNotFoundException(String.format("Named query (%1$s) does not exist!", key));
     }
   }
 
-  protected Region<String, String> getQueryStore(final String namePath) {
+  Region<String, String> getQueryStore(final String namePath) {
     return ValidationUtils.returnValueThrowOnNull(getCache().<String, String>getRegion(namePath),
         new GemfireRestException(String.format("Query store does not exist!", namePath)));
   }
@@ -436,8 +425,7 @@ public abstract class AbstractBaseController {
     }
   }
 
-  protected void updateNamedQuery(final String regionNamePath, final String key,
-      final String value) {
+  void updateNamedQuery(final String regionNamePath, final String key, final String value) {
     try {
       getQueryStore(regionNamePath).put(key, value);
     } catch (NullPointerException npe) {
@@ -457,8 +445,7 @@ public abstract class AbstractBaseController {
   }
 
   @SuppressWarnings("unchecked")
-  protected <T> T createNamedQuery(final String regionNamePath, final String key,
-      final String value) {
+  <T> T createNamedQuery(final String regionNamePath, final String key, final String value) {
     try {
       return (T) getQueryStore(regionNamePath).putIfAbsent(key, value);
     } catch (UnsupportedOperationException use) {
@@ -482,7 +469,7 @@ public abstract class AbstractBaseController {
     }
   }
 
-  protected void putPdxValues(final String regionNamePath, final Map<Object, PdxInstance> map) {
+  private void putPdxValues(final String regionNamePath, final Map<Object, PdxInstance> map) {
     try {
       getRegion(regionNamePath).putAll(map);
     } catch (LowMemoryException lme) {
@@ -490,7 +477,7 @@ public abstract class AbstractBaseController {
     }
   }
 
-  protected void putValues(final String regionNamePath, final Map<Object, Object> values) {
+  private void putValues(final String regionNamePath, final Map<Object, Object> values) {
     getRegion(regionNamePath).putAll(values);
   }
 
@@ -510,7 +497,7 @@ public abstract class AbstractBaseController {
   }
 
   @SuppressWarnings("unchecked")
-  protected <T> T postValue(final String regionNamePath, final Object key, final Object value) {
+  <T> T postValue(final String regionNamePath, final Object key, final Object value) {
     try {
       return (T) getRegion(regionNamePath).putIfAbsent(key, value);
     } catch (UnsupportedOperationException use) {
@@ -561,11 +548,11 @@ public abstract class AbstractBaseController {
     return actualValue;
   }
 
-  protected String generateKey(final String existingKey) {
+  String generateKey(final String existingKey) {
     return generateKey(existingKey, null);
   }
 
-  protected String generateKey(final String existingKey, final Object domainObject) {
+  private String generateKey(final String existingKey, final Object domainObject) {
     Object domainObjectId = IdentifiableUtils.getId(domainObject);
     String newKey;
 
@@ -597,7 +584,7 @@ public abstract class AbstractBaseController {
     return newKey;
   }
 
-  protected String decode(final String value, final String encoding) {
+  private String decode(final String value, final String encoding) {
     try {
       return URLDecoder.decode(value, encoding);
     } catch (UnsupportedEncodingException e) {
@@ -612,19 +599,17 @@ public abstract class AbstractBaseController {
             String.format("The Region identified by name (%1$s) could not be found!", namePath)));
   }
 
-  protected void checkForKeyExist(String region, String key) {
+  private void checkForKeyExist(String region, String key) {
     if (!getRegion(region).containsKey(key)) {
       throw new ResourceNotFoundException(
           String.format("Key (%1$s) does not exist for region (%2$s) in cache!", key, region));
     }
   }
 
-  protected List<String> checkForMultipleKeysExist(String region, String... keys) {
+  List<String> checkForMultipleKeysExist(String region, String... keys) {
     List<String> unknownKeys = new ArrayList<String>();
     for (int index = 0; index < keys.length; index++) {
       if (!getRegion(region).containsKey(keys[index])) {
-        // throw new ResourceNotFoundException(String.format("Key [(%1$s)] does not exist for region
-        // [(%2$s)] in cache!", key, region));
         unknownKeys.add(keys[index]);
       }
     }
@@ -664,11 +649,11 @@ public abstract class AbstractBaseController {
     return entries.values();
   }
 
-  protected void deleteValue(final String regionNamePath, final Object key) {
+  private void deleteValue(final String regionNamePath, final Object key) {
     getRegion(regionNamePath).remove(key);
   }
 
-  protected void deleteValues(final String regionNamePath, final Object... keys) {
+  void deleteValues(final String regionNamePath, final Object... keys) {
     // Check whether all keys exist in cache or not
     for (final Object key : keys) {
       checkForKeyExist(regionNamePath, key.toString());
@@ -679,7 +664,7 @@ public abstract class AbstractBaseController {
     }
   }
 
-  protected void deleteValues(String regionNamePath) {
+  void deleteValues(String regionNamePath) {
     try {
       getRegion(regionNamePath).clear();
     } catch (UnsupportedOperationException ue) {
@@ -694,7 +679,7 @@ public abstract class AbstractBaseController {
   }
 
   @SuppressWarnings("unchecked")
-  protected <T> T introspectAndConvert(final T value) {
+  private <T> T introspectAndConvert(final T value) {
     if (value instanceof Map) {
       final Map rawDataBinding = (Map) value;
 
@@ -736,15 +721,15 @@ public abstract class AbstractBaseController {
     return value;
   }
 
-  protected String convertErrorAsJson(String errorMessage) {
+  String convertErrorAsJson(String errorMessage) {
     return ("{" + "\"message\"" + ":" + "\"" + errorMessage + "\"" + "}");
   }
 
-  protected String convertErrorAsJson(Throwable t) {
+  String convertErrorAsJson(Throwable t) {
     return String.format("{\"message\" : \"%1$s\"}", t.getMessage());
   }
 
-  protected Map<?, ?> convertJsonToMap(final String jsonString) {
+  private Map<?, ?> convertJsonToMap(final String jsonString) {
     Map<String, String> map = new HashMap<String, String>();
 
     // convert JSON string to Map
@@ -762,11 +747,11 @@ public abstract class AbstractBaseController {
     return map;
   }
 
-  protected Object jsonToObject(final String jsonString) {
+  private Object jsonToObject(final String jsonString) {
     return introspectAndConvert(convertJsonToMap(jsonString));
   }
 
-  protected Object[] jsonToObjectArray(final String arguments) {
+  Object[] jsonToObjectArray(final String arguments) {
     final JSONTypes jsonType = validateJsonAndFindType(arguments);
     if (JSONTypes.JSON_ARRAY.equals(jsonType)) {
       try {
@@ -787,8 +772,8 @@ public abstract class AbstractBaseController {
     }
   }
 
-  public ResponseEntity<String> updateSingleKey(final String region, final String key,
-      final String json, final String opValue) {
+  ResponseEntity<String> updateSingleKey(final String region, final String key, final String json,
+      final String opValue) {
 
     final JSONTypes jsonType = validateJsonAndFindType(json);
 
@@ -822,7 +807,7 @@ public abstract class AbstractBaseController {
   }
 
 
-  public ResponseEntity<String> updateMultipleKeys(final String region, final String[] keys,
+  ResponseEntity<String> updateMultipleKeys(final String region, final String[] keys,
       final String json) {
 
     JSONArray jsonArr = null;
@@ -862,7 +847,7 @@ public abstract class AbstractBaseController {
     return new ResponseEntity<String>(headers, HttpStatus.OK);
   }
 
-  public JSONTypes validateJsonAndFindType(String json) {
+  JSONTypes validateJsonAndFindType(String json) {
     try {
       Object jsonObj = new JSONTokener(json).nextValue();
 
@@ -927,17 +912,16 @@ public abstract class AbstractBaseController {
   }
 
   protected Set<DistributedMember> getMembers(final String... memberIdNames) {
-
     ValidationUtils.returnValueThrowOnNull(memberIdNames,
         new GemfireRestException("No member found to run function"));
-    final Set<DistributedMember> targetedMembers =
-        new HashSet<DistributedMember>(ArrayUtils.length(memberIdNames));
+    final Set<DistributedMember> targetedMembers = new HashSet<>(ArrayUtils.length(memberIdNames));
     final List<String> memberIdNameList = Arrays.asList(memberIdNames);
-    GemFireCacheImpl c = (GemFireCacheImpl) getCache();
-    Set<DistributedMember> distMembers = c.getDistributedSystem().getAllOtherMembers();
+
+    InternalCache cache = getCache();
+    Set<DistributedMember> distMembers = cache.getDistributedSystem().getAllOtherMembers();
 
     // Add the local node to list
-    distMembers.add(c.getDistributedSystem().getDistributedMember());
+    distMembers.add(cache.getDistributedSystem().getDistributedMember());
     for (DistributedMember member : distMembers) {
       if (memberIdNameList.contains(member.getId())
           || memberIdNameList.contains(member.getName())) {
@@ -947,10 +931,10 @@ public abstract class AbstractBaseController {
     return targetedMembers;
   }
 
-  protected Set<DistributedMember> getAllMembersInDS() {
-    GemFireCacheImpl c = (GemFireCacheImpl) getCache();
-    Set<DistributedMember> distMembers = c.getDistributedSystem().getAllOtherMembers();
-    final Set<DistributedMember> targetedMembers = new HashSet<DistributedMember>();
+  Set<DistributedMember> getAllMembersInDS() {
+    InternalCache cache = getCache();
+    Set<DistributedMember> distMembers = cache.getDistributedSystem().getAllOtherMembers();
+    final Set<DistributedMember> targetedMembers = new HashSet<>();
 
     // find valid data nodes, i.e non locator, non-admin, non-loner nodes
     for (DistributedMember member : distMembers) {
@@ -960,7 +944,7 @@ public abstract class AbstractBaseController {
       }
     }
     // Add the local node to list
-    targetedMembers.add(c.getDistributedSystem().getDistributedMember());
+    targetedMembers.add(cache.getDistributedSystem().getDistributedMember());
     return targetedMembers;
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/47d8c820/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/BaseControllerAdvice.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/BaseControllerAdvice.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/BaseControllerAdvice.java
index 8939542..751e6a0 100644
--- a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/BaseControllerAdvice.java
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/BaseControllerAdvice.java
@@ -12,16 +12,11 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.rest.internal.web.controllers;
 
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.rest.internal.web.exception.DataTypeNotSupportedException;
-import org.apache.geode.rest.internal.web.exception.GemfireRestException;
-import org.apache.geode.rest.internal.web.exception.MalformedJsonException;
-import org.apache.geode.rest.internal.web.exception.RegionNotFoundException;
-import org.apache.geode.rest.internal.web.exception.ResourceNotFoundException;
-import org.apache.geode.security.NotAuthorizedException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+
 import org.apache.logging.log4j.Logger;
 import org.springframework.http.HttpStatus;
 import org.springframework.security.access.AccessDeniedException;
@@ -31,24 +26,26 @@ import org.springframework.web.bind.annotation.ExceptionHandler;
 import org.springframework.web.bind.annotation.ResponseBody;
 import org.springframework.web.bind.annotation.ResponseStatus;
 
-import java.io.PrintWriter;
-import java.io.StringWriter;
-
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.rest.internal.web.exception.DataTypeNotSupportedException;
+import org.apache.geode.rest.internal.web.exception.GemfireRestException;
+import org.apache.geode.rest.internal.web.exception.MalformedJsonException;
+import org.apache.geode.rest.internal.web.exception.RegionNotFoundException;
+import org.apache.geode.rest.internal.web.exception.ResourceNotFoundException;
+import org.apache.geode.security.NotAuthorizedException;
 
 /**
  * The CrudControllerAdvice class handles exception thrown while serving the REST request
- * <p/>
- * 
+ *
  * @since GemFire 8.0
  */
-
 @ControllerAdvice
 @SuppressWarnings("unused")
 public class BaseControllerAdvice extends AbstractBaseController {
 
   private static final Logger logger = LogService.getLogger();
 
-  protected static final String REST_API_VERSION = "/v1";
+  private static final String REST_API_VERSION = "/v1";
 
   @Override
   protected String getRestApiVersion() {
@@ -186,4 +183,3 @@ public class BaseControllerAdvice extends AbstractBaseController {
   }
 
 }
-

http://git-wip-us.apache.org/repos/asf/geode/blob/47d8c820/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java
index 62ce860..0449a45 100644
--- a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/CommonCrudController.java
@@ -14,9 +14,24 @@
  */
 package org.apache.geode.rest.internal.web.controllers;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
+import org.apache.logging.log4j.Logger;
+import org.json.JSONException;
+import org.springframework.http.HttpHeaders;
+import org.springframework.http.HttpStatus;
+import org.springframework.http.MediaType;
+import org.springframework.http.ResponseEntity;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+
 import org.apache.geode.cache.LowMemoryException;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.execute.Execution;
@@ -29,20 +44,6 @@ import org.apache.geode.rest.internal.web.controllers.support.RestServersResultC
 import org.apache.geode.rest.internal.web.exception.GemfireRestException;
 import org.apache.geode.rest.internal.web.util.ArrayUtils;
 import org.apache.geode.rest.internal.web.util.JSONUtils;
-import org.apache.logging.log4j.Logger;
-import org.json.JSONException;
-import org.springframework.http.HttpHeaders;
-import org.springframework.http.HttpStatus;
-import org.springframework.http.MediaType;
-import org.springframework.http.ResponseEntity;
-import org.springframework.security.access.prepost.PreAuthorize;
-import org.springframework.web.bind.annotation.PathVariable;
-import org.springframework.web.bind.annotation.RequestMapping;
-import org.springframework.web.bind.annotation.RequestMethod;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
 
 /**
  * The CommonCrudController serves REST Requests related to listing regions, listing keys in region,

http://git-wip-us.apache.org/repos/asf/geode/blob/47d8c820/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/FunctionAccessController.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/FunctionAccessController.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/FunctionAccessController.java
index 80996c3..473b8b9 100644
--- a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/FunctionAccessController.java
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/FunctionAccessController.java
@@ -12,24 +12,17 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.rest.internal.web.controllers;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
-import org.apache.geode.cache.LowMemoryException;
-import org.apache.geode.cache.execute.Execution;
-import org.apache.geode.cache.execute.Function;
-import org.apache.geode.cache.execute.FunctionException;
-import org.apache.geode.cache.execute.FunctionService;
-import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.internal.cache.execute.NoResult;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.rest.internal.web.exception.GemfireRestException;
-import org.apache.geode.rest.internal.web.util.ArrayUtils;
-import org.apache.geode.rest.internal.web.util.JSONUtils;
 import org.apache.logging.log4j.Logger;
 import org.json.JSONException;
 import org.springframework.http.HttpHeaders;
@@ -47,10 +40,17 @@ import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 import org.springframework.web.bind.annotation.ResponseStatus;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import org.apache.geode.cache.LowMemoryException;
+import org.apache.geode.cache.execute.Execution;
+import org.apache.geode.cache.execute.Function;
+import org.apache.geode.cache.execute.FunctionException;
+import org.apache.geode.cache.execute.FunctionService;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.internal.cache.execute.NoResult;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.rest.internal.web.exception.GemfireRestException;
+import org.apache.geode.rest.internal.web.util.ArrayUtils;
+import org.apache.geode.rest.internal.web.util.JSONUtils;
 
 /**
  * The FunctionsController class serving REST Requests related to the function execution
@@ -58,20 +58,18 @@ import java.util.Set;
  * @see org.springframework.stereotype.Controller
  * @since GemFire 8.0
  */
-
 @Controller("functionController")
 @Api(value = "functions", description = "Rest api for gemfire function execution")
 @RequestMapping(FunctionAccessController.REST_API_VERSION + "/functions")
 @SuppressWarnings("unused")
 public class FunctionAccessController extends AbstractBaseController {
   // Constant String value indicating the version of the REST API.
-  protected static final String REST_API_VERSION = "/v1";
+  static final String REST_API_VERSION = "/v1";
   private static final Logger logger = LogService.getLogger();
 
   /**
    * Gets the version of the REST API implemented by this @Controller.
-   * <p>
-   * 
+   *
    * @return a String indicating the REST API version.
    */
   @Override
@@ -256,4 +254,3 @@ public class FunctionAccessController extends AbstractBaseController {
     }
   }
 }
-

http://git-wip-us.apache.org/repos/asf/geode/blob/47d8c820/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/PdxBasedCrudController.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/PdxBasedCrudController.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/PdxBasedCrudController.java
index 3b08c5f..a6f10b1 100644
--- a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/PdxBasedCrudController.java
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/PdxBasedCrudController.java
@@ -14,16 +14,14 @@
  */
 package org.apache.geode.rest.internal.web.controllers;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.rest.internal.web.controllers.support.JSONTypes;
-import org.apache.geode.rest.internal.web.controllers.support.RegionData;
-import org.apache.geode.rest.internal.web.controllers.support.RegionEntryData;
-import org.apache.geode.rest.internal.web.exception.ResourceNotFoundException;
-import org.apache.geode.rest.internal.web.util.ArrayUtils;
 import org.apache.logging.log4j.Logger;
 import org.springframework.http.HttpHeaders;
 import org.springframework.http.HttpStatus;
@@ -38,19 +36,20 @@ import org.springframework.web.bind.annotation.RequestMapping;
 import org.springframework.web.bind.annotation.RequestMethod;
 import org.springframework.web.bind.annotation.RequestParam;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.rest.internal.web.controllers.support.JSONTypes;
+import org.apache.geode.rest.internal.web.controllers.support.RegionData;
+import org.apache.geode.rest.internal.web.controllers.support.RegionEntryData;
+import org.apache.geode.rest.internal.web.exception.ResourceNotFoundException;
+import org.apache.geode.rest.internal.web.util.ArrayUtils;
 
 /**
  * The PdxBasedCrudController class serving REST Requests related to the REST CRUD operation on
  * region
- * <p/>
- * 
+ *
  * @see org.springframework.stereotype.Controller
  * @since GemFire 8.0
  */
-
 @Controller("pdxCrudController")
 @Api(value = "region", description = "region CRUD operations")
 @RequestMapping(PdxBasedCrudController.REST_API_VERSION)
@@ -59,9 +58,9 @@ public class PdxBasedCrudController extends CommonCrudController {
 
   private static final Logger logger = LogService.getLogger();
 
-  protected static final String REST_API_VERSION = "/v1";
+  static final String REST_API_VERSION = "/v1";
 
-  protected static final String DEFAULT_GETALL_RESULT_LIMIT = "50";
+  private static final String DEFAULT_GETALL_RESULT_LIMIT = "50";
 
   @Override
   protected String getRestApiVersion() {
@@ -76,7 +75,6 @@ public class PdxBasedCrudController extends CommonCrudController {
    * @param json JSON document that is stored against the key
    * @return JSON document
    */
-
   @RequestMapping(method = RequestMethod.POST, value = "/{region}",
       consumes = MediaType.APPLICATION_JSON_VALUE, produces = {MediaType.APPLICATION_JSON_VALUE})
   @ApiOperation(value = "create entry", notes = "Create (put-if-absent) data in region",

http://git-wip-us.apache.org/repos/asf/geode/blob/47d8c820/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/QueryAccessController.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/QueryAccessController.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/QueryAccessController.java
index b00a7aa..8007491 100644
--- a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/QueryAccessController.java
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/QueryAccessController.java
@@ -12,28 +12,14 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.rest.internal.web.controllers;
 
+import java.util.concurrent.ConcurrentHashMap;
+
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.query.FunctionDomainException;
-import org.apache.geode.cache.query.NameResolutionException;
-import org.apache.geode.cache.query.Query;
-import org.apache.geode.cache.query.QueryExecutionLowMemoryException;
-import org.apache.geode.cache.query.QueryExecutionTimeoutException;
-import org.apache.geode.cache.query.QueryInvalidException;
-import org.apache.geode.cache.query.QueryInvocationTargetException;
-import org.apache.geode.cache.query.TypeMismatchException;
-import org.apache.geode.cache.query.internal.DefaultQuery;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.rest.internal.web.exception.GemfireRestException;
-import org.apache.geode.rest.internal.web.exception.ResourceNotFoundException;
-import org.apache.geode.rest.internal.web.util.JSONUtils;
-import org.apache.geode.rest.internal.web.util.ValidationUtils;
 import org.apache.logging.log4j.Logger;
 import org.springframework.http.HttpHeaders;
 import org.springframework.http.HttpStatus;
@@ -49,17 +35,28 @@ import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 import org.springframework.web.bind.annotation.ResponseStatus;
 
-import java.util.concurrent.ConcurrentHashMap;
-
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.query.FunctionDomainException;
+import org.apache.geode.cache.query.NameResolutionException;
+import org.apache.geode.cache.query.Query;
+import org.apache.geode.cache.query.QueryExecutionLowMemoryException;
+import org.apache.geode.cache.query.QueryExecutionTimeoutException;
+import org.apache.geode.cache.query.QueryInvalidException;
+import org.apache.geode.cache.query.QueryInvocationTargetException;
+import org.apache.geode.cache.query.TypeMismatchException;
+import org.apache.geode.cache.query.internal.DefaultQuery;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.rest.internal.web.exception.GemfireRestException;
+import org.apache.geode.rest.internal.web.exception.ResourceNotFoundException;
+import org.apache.geode.rest.internal.web.util.JSONUtils;
+import org.apache.geode.rest.internal.web.util.ValidationUtils;
 
 /**
  * The QueryingController class serves all HTTP REST requests related to the gemfire querying
- * <p/>
- * 
+ *
  * @see org.springframework.stereotype.Controller
  * @since GemFire 8.0
  */
-
 @Controller("queryController")
 @Api(value = "queries", description = "Rest api for geode query execution",
     produces = MediaType.APPLICATION_JSON_VALUE)
@@ -69,17 +66,16 @@ public class QueryAccessController extends AbstractBaseController {
 
   private static final Logger logger = LogService.getLogger();
 
-  protected static final String PARAMETERIZED_QUERIES_REGION = "__ParameterizedQueries__";
+  private static final String PARAMETERIZED_QUERIES_REGION = "__ParameterizedQueries__";
 
   private final ConcurrentHashMap<String, DefaultQuery> compiledQueries = new ConcurrentHashMap<>();
 
   // Constant String value indicating the version of the REST API.
-  protected static final String REST_API_VERSION = "/v1";
+  static final String REST_API_VERSION = "/v1";
 
   /**
    * Gets the version of the REST API implemented by this @Controller.
-   * <p/>
-   * 
+   *
    * @return a String indicating the REST API version.
    */
   @Override
@@ -324,7 +320,6 @@ public class QueryAccessController extends AbstractBaseController {
     logger.debug("Updating a named, parametrized Query ({}) with ID ({})...", oqlStatement,
         queryId);
 
-
     // update the OQL statement with 'queryId' as the Key into the hidden, Parametrized Queries
     // Region...
     checkForQueryIdExist(PARAMETERIZED_QUERIES_REGION, queryId);
@@ -334,7 +329,6 @@ public class QueryAccessController extends AbstractBaseController {
     return new ResponseEntity<>(HttpStatus.OK);
   }
 
-  // delete named, parametrized query
   /**
    * Delete named, parametrized Query
    * 
@@ -360,4 +354,3 @@ public class QueryAccessController extends AbstractBaseController {
   }
 
 }
-

http://git-wip-us.apache.org/repos/asf/geode/blob/47d8c820/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/support/CacheProvider.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/support/CacheProvider.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/support/CacheProvider.java
new file mode 100644
index 0000000..19d2755
--- /dev/null
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/support/CacheProvider.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.rest.internal.web.controllers.support;
+
+import org.apache.geode.internal.cache.InternalCache;
+
+public interface CacheProvider {
+
+  InternalCache getInternalCache();
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/47d8c820/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/support/CacheProviderImpl.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/support/CacheProviderImpl.java b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/support/CacheProviderImpl.java
new file mode 100644
index 0000000..8c00923
--- /dev/null
+++ b/geode-web-api/src/main/java/org/apache/geode/rest/internal/web/controllers/support/CacheProviderImpl.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.rest.internal.web.controllers.support;
+
+import org.springframework.stereotype.Component;
+
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
+
+@Component("cacheProvider")
+public class CacheProviderImpl implements CacheProvider {
+
+  @Override
+  public InternalCache getInternalCache() {
+    return GemFireCacheImpl.getExisting();
+  }
+}


[02/13] geode git commit: GEODE-2806: if the batch is dispatched, even the bucket is no longer primary, the batch should still be deleted as planned.

Posted by kl...@apache.org.
GEODE-2806: if the batch is dispatched, even the bucket is no longer primary, the batch should still be deleted as planned.


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

Branch: refs/heads/feature/GEODE-2632-6-1
Commit: 0862174c30cad1536f2c105b783653bd0d4344e8
Parents: 50686b0
Author: zhouxh <gz...@pivotal.io>
Authored: Fri Apr 21 09:57:05 2017 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Fri Apr 21 09:58:19 2017 -0700

----------------------------------------------------------------------
 .../parallel/ParallelGatewaySenderQueue.java    | 58 ++++++++++----------
 1 file changed, 28 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/0862174c/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
index cf4c5a9..9696b90 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
@@ -1104,38 +1104,36 @@ public class ParallelGatewaySenderQueue implements RegionQueue {
 
   private void destroyEventFromQueue(PartitionedRegion prQ, int bucketId, Object key) {
     boolean isPrimary = prQ.getRegionAdvisor().getBucketAdvisor(bucketId).isPrimary();
-    if (isPrimary) {
-      BucketRegionQueue brq = getBucketRegionQueueByBucketId(prQ, bucketId);
-      // TODO : Kishor : Make sure we dont need to initalize a bucket
-      // before destroying a key from it
-      try {
-        if (brq != null) {
-          brq.destroyKey(key);
-        }
-        stats.decQueueSize();
-      } catch (EntryNotFoundException e) {
-        if (!this.sender.isBatchConflationEnabled() && logger.isDebugEnabled()) {
-          logger.debug(
-              "ParallelGatewaySenderQueue#remove: Got EntryNotFoundException while removing key {} for {} for bucket = {} for GatewaySender {}",
-              key, this, bucketId, this.sender);
-        }
-      } catch (ForceReattemptException e) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("Bucket :{} moved to other member", bucketId);
-        }
-      } catch (PrimaryBucketException e) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("Primary bucket :{} moved to other member", bucketId);
-        }
-      } catch (RegionDestroyedException e) {
-        if (logger.isDebugEnabled()) {
-          logger.debug(
-              "Caught RegionDestroyedException attempting to remove key {} from bucket {} in {}",
-              key, bucketId, prQ.getFullPath());
-        }
+    BucketRegionQueue brq = getBucketRegionQueueByBucketId(prQ, bucketId);
+    // TODO : Kishor : Make sure we dont need to initalize a bucket
+    // before destroying a key from it
+    try {
+      if (brq != null) {
+        brq.destroyKey(key);
+      }
+      stats.decQueueSize();
+    } catch (EntryNotFoundException e) {
+      if (!this.sender.isBatchConflationEnabled() && logger.isDebugEnabled()) {
+        logger.debug(
+            "ParallelGatewaySenderQueue#remove: Got EntryNotFoundException while removing key {} for {} for bucket = {} for GatewaySender {}",
+            key, this, bucketId, this.sender);
+      }
+    } catch (ForceReattemptException e) {
+      if (logger.isDebugEnabled()) {
+        logger.debug("Bucket :{} moved to other member", bucketId);
+      }
+    } catch (PrimaryBucketException e) {
+      if (logger.isDebugEnabled()) {
+        logger.debug("Primary bucket :{} moved to other member", bucketId);
+      }
+    } catch (RegionDestroyedException e) {
+      if (logger.isDebugEnabled()) {
+        logger.debug(
+            "Caught RegionDestroyedException attempting to remove key {} from bucket {} in {}", key,
+            bucketId, prQ.getFullPath());
       }
-      addRemovedEvent(prQ, bucketId, key);
     }
+    addRemovedEvent(prQ, bucketId, key);
   }
 
   public void resetLastPeeked() {


[09/13] geode git commit: GEODE-2809 Geode docs: Clarify SSL setup for client This closes #473

Posted by kl...@apache.org.
GEODE-2809 Geode docs: Clarify SSL setup for client
This closes #473


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

Branch: refs/heads/feature/GEODE-2632-6-1
Commit: 9f8ba8de90e018c5c20f960447fd9b201e8fbcf1
Parents: 6eb9ff3
Author: Dave Barnes <db...@pivotal.io>
Authored: Fri Apr 21 11:54:33 2017 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Mon Apr 24 12:16:15 2017 -0700

----------------------------------------------------------------------
 .../managing/security/implementing_ssl.html.md.erb    | 14 +++++++++-----
 1 file changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/9f8ba8de/geode-docs/managing/security/implementing_ssl.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/managing/security/implementing_ssl.html.md.erb b/geode-docs/managing/security/implementing_ssl.html.md.erb
index c19b6d9..4894de7 100644
--- a/geode-docs/managing/security/implementing_ssl.html.md.erb
+++ b/geode-docs/managing/security/implementing_ssl.html.md.erb
@@ -66,13 +66,13 @@ You can use Geode configuration properties to enable or disable SSL, to identify
 protocols, and to provide the location and credentials for key and trust stores.
 
 <dt>**ssl-enabled-components**</dt>
-<dd>list of components for which to enable SSL. "all" or comma-separated list of components</dd>
+<dd>List of components for which to enable SSL. Component list can be "all" or a comma-separated list of components.</dd>
 
 <dt>**ssl-require-authentication**</dt>
-<dd>Requires two-way authentication, applies to all components except web. boolean - if true (the default), two-way authentication is required.</dd>
+<dd>Requires two-way authentication, applies to all components except web. Boolean - if true (the default), two-way authentication is required.</dd>
 
 <dt>**ssl-web-require-authentication**</dt>
-<dd>Requires two-way authentication for web component. boolean - if true, two-way authentication is required. Default is false (one-way authentication only).</dd>
+<dd>Requires two-way authentication for web component. Boolean - if true, two-way authentication is required. Default is false (one-way authentication only).</dd>
 
 <dt>**ssl-default-alias**</dt>
 <dd>A server uses one key store to hold its SSL certificates. All components on that server can share a
@@ -151,9 +151,13 @@ ssl-default-alias=Locator-Cert
  
 **Client properties**
 
-The client's trust store must trust both locator and server certificates.
+On the client, the list of enabled components reflects the server's configuration so the client
+knows how it is expected to communicate with (for example) servers and locators.  Paths to keystore
+and truststore are local to the client.
 
-Since the client did not specify a certificate alias, SSL will use the default certificate in its key store.
+In this example, the client's trust store must trust both locator and server certificates.  Since
+the client does not specify a certificate alias, SSL will use the default certificate in its key
+store.
 
 ``` pre
 ssl-enabled-components=server,locator