You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2015/10/26 19:28:30 UTC

[07/50] [abbrv] incubator-geode git commit: GEODE-346: Split into UnitTests and IntegrationTests

GEODE-346: Split into UnitTests and IntegrationTests

Extract IntegrationTests from AutoBalancerJUnitTest and move some
common code to setup and teardown methods. Fix GEODE-346 by making
teardown wait for HostStatSampler statThread to terminate.

Isolate test with sleep to its own test class and document. The next
step should involve refactoring of AutoBalancer to break dependencies
to allow for correct mocking.

Add Awaitility version 1.6.5 to dependencies (GEODE-217).

Add check to see if HostStatSampler statThread is alive or not.


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

Branch: refs/heads/develop
Commit: f801d1cfc654772fc399f9baae22d32abbe50ac6
Parents: a23c33c
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Oct 20 15:44:40 2015 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Oct 20 15:59:01 2015 -0700

----------------------------------------------------------------------
 build.gradle                                    |   1 +
 .../gemfire/internal/HostStatSampler.java       |   4 +-
 ...erAuditorInvocationIntegrationJUnitTest.java |  80 ++++++
 .../util/AutoBalancerIntegrationJUnitTest.java  | 279 +++++++++++++++++++
 .../cache/util/AutoBalancerJUnitTest.java       | 273 +-----------------
 5 files changed, 375 insertions(+), 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f801d1cf/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index f2aa561..33e73a5 100755
--- a/build.gradle
+++ b/build.gradle
@@ -285,6 +285,7 @@ subprojects {
     compile 'org.springframework:spring-webmvc:3.2.12.RELEASE'
 
     testCompile 'com.github.stefanbirkner:system-rules:1.12.1'
+    testCompile 'com.jayway.awaitility:awaitility:1.6.5'
     testCompile 'edu.umd.cs.mtc:multithreadedtc:1.01'
     testCompile 'junit:junit:4.12'
     testCompile 'org.assertj:assertj-core:2.1.0'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f801d1cf/gemfire-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java
index eadfc34..91b2abe 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/HostStatSampler.java
@@ -327,7 +327,9 @@ public abstract class HostStatSampler
   }
   
   public final boolean isAlive() {
-    return statThread.isAlive();
+    synchronized (HostStatSampler.class) {
+      return statThread != null && statThread.isAlive();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f801d1cf/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerAuditorInvocationIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerAuditorInvocationIntegrationJUnitTest.java b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerAuditorInvocationIntegrationJUnitTest.java
new file mode 100755
index 0000000..bd6a3ff
--- /dev/null
+++ b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerAuditorInvocationIntegrationJUnitTest.java
@@ -0,0 +1,80 @@
+package com.gemstone.gemfire.cache.util;
+
+import static org.junit.Assert.*;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.jmock.Expectations;
+import org.jmock.Mockery;
+import org.jmock.lib.concurrent.Synchroniser;
+import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.util.AutoBalancer.OOBAuditor;
+import com.gemstone.gemfire.cache.util.AutoBalancer.TimeProvider;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * IntegrationTest for AuditorInvocation in AutoBalancer. 
+ * 
+ * <p>AutoBalancer should:<br>
+ * 1) be refactored to extract out all inner-classes and inner-interfaces<br>
+ * 2) have constructor changed to accept every collaborator as an argument<br>
+ * 3) then this test can correctly use mocking without any real threads to wait on
+ * 
+ * <p>Extracted from AutoBalancerJUnitTest
+ */
+@Category(IntegrationTest.class)
+public class AutoBalancerAuditorInvocationIntegrationJUnitTest {
+
+  Mockery mockContext;
+
+  @Before
+  public void setupMock() {
+    mockContext = new Mockery() {
+      {
+        setImposteriser(ClassImposteriser.INSTANCE);
+        setThreadingPolicy(new Synchroniser());
+      }
+    };
+  }
+
+  @After
+  public void validateMock() {
+    mockContext.assertIsSatisfied();
+    mockContext = null;
+  }
+
+  @Test
+  public void testAuditorInvocation() throws InterruptedException {
+    int count = 0;
+
+    final OOBAuditor mockAuditor = mockContext.mock(OOBAuditor.class);
+    final TimeProvider mockClock = mockContext.mock(TimeProvider.class);
+    mockContext.checking(new Expectations() {
+      {
+        oneOf(mockAuditor).init(with(any(Properties.class)));
+        exactly(2).of(mockAuditor).execute();
+        allowing(mockClock).currentTimeMillis();
+        will(returnValue(950L));
+      }
+    });
+
+    Properties props = AutoBalancerJUnitTest.getBasicConfig();
+
+    assertEquals(0, count);
+    AutoBalancer autoR = new AutoBalancer();
+    autoR.setOOBAuditor(mockAuditor);
+    autoR.setTimeProvider(mockClock);
+
+    // the trigger should get invoked after 50 milliseconds
+    autoR.init(props);
+    
+    // TODO: this sleep should NOT be here -- use Awaitility to await a condition instead or use mocking to avoid this altogether
+    TimeUnit.MILLISECONDS.sleep(120); // removal causes failure in validateMock
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f801d1cf/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerIntegrationJUnitTest.java b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerIntegrationJUnitTest.java
new file mode 100755
index 0000000..38b7bf9
--- /dev/null
+++ b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerIntegrationJUnitTest.java
@@ -0,0 +1,279 @@
+package com.gemstone.gemfire.cache.util;
+
+import static com.jayway.awaitility.Awaitility.*;
+import static java.util.concurrent.TimeUnit.*;
+import static org.junit.Assert.*;
+import static org.hamcrest.Matchers.*;
+
+import java.io.ByteArrayInputStream;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.jmock.Expectations;
+import org.jmock.Mockery;
+import org.jmock.api.Invocation;
+import org.jmock.lib.action.CustomAction;
+import org.jmock.lib.concurrent.Synchroniser;
+import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.GemFireConfigException;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.util.AutoBalancer.CacheOperationFacade;
+import com.gemstone.gemfire.cache.util.AutoBalancer.GeodeCacheFacade;
+import com.gemstone.gemfire.distributed.DistributedLockService;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.locks.DLockService;
+import com.gemstone.gemfire.internal.HostStatSampler;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.PRHARedundancyProvider;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.partitioned.InternalPRInfo;
+import com.gemstone.gemfire.internal.cache.partitioned.LoadProbe;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * IntegrationTests for AutoBalancer that include usage of Cache, StatSampler 
+ * and DistributedLockService. Some collaborators may be mocked while others
+ * are real.
+ * 
+ * <p>Extracted from AutoBalancerJUnitTest
+ */
+@Category(IntegrationTest.class)
+public class AutoBalancerIntegrationJUnitTest {
+  
+  private static final int TIMEOUT_SECONDS = 5;
+
+  private GemFireCacheImpl cache;
+  private Mockery mockContext;
+
+  @Before
+  public void setupMock() {
+    mockContext = new Mockery() {
+      {
+        setImposteriser(ClassImposteriser.INSTANCE);
+        setThreadingPolicy(new Synchroniser());
+      }
+    };
+  }
+  
+  @Before
+  public void setUpCacheAndDLS() {
+    cache = createBasicCache();
+  }
+
+  @After
+  public void destroyCacheAndDLS() {
+    if (DLockService.getServiceNamed(AutoBalancer.AUTO_BALANCER_LOCK_SERVICE_NAME) != null) {
+      DLockService.destroy(AutoBalancer.AUTO_BALANCER_LOCK_SERVICE_NAME);
+    }
+
+    if (cache != null && !cache.isClosed()) {
+      try {
+        final HostStatSampler statSampler = ((InternalDistributedSystem)cache.getDistributedSystem()).getStatSampler();
+        cache.close();
+        // wait for the stat sampler to stand down
+        await().atMost(TIMEOUT_SECONDS, SECONDS).until(isAlive(statSampler), equalTo(false));
+      } finally {
+        cache = null;
+      }
+    }
+  }
+  
+  @After
+  public void validateMock() {
+    mockContext.assertIsSatisfied();
+    mockContext = null;
+  }
+
+  @Test
+  public void testAutoRebalaceStatsOnLockSuccess() throws InterruptedException {
+    final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
+    mockContext.checking(new Expectations() {
+      {
+        oneOf(mockCacheFacade).acquireAutoBalanceLock();
+        will(returnValue(true));
+        oneOf(mockCacheFacade).incrementAttemptCounter();
+        will(new CustomAction("increment stat") {
+          public Object invoke(Invocation invocation) throws Throwable {
+            new GeodeCacheFacade().incrementAttemptCounter();
+            return null;
+          }
+        });
+        allowing(mockCacheFacade);
+      }
+    });
+
+    assertEquals(0, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
+    AutoBalancer balancer = new AutoBalancer();
+    balancer.setCacheOperationFacade(mockCacheFacade);
+    balancer.getOOBAuditor().execute();
+    
+    assertEquals(1, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
+  }
+
+  @Test
+  public void testAutoRebalaceStatsOnLockFailure() throws InterruptedException {
+    final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
+    mockContext.checking(new Expectations() {
+      {
+        oneOf(mockCacheFacade).acquireAutoBalanceLock();
+        will(returnValue(false));
+      }
+    });
+
+    assertEquals(0, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
+    AutoBalancer balancer = new AutoBalancer();
+    balancer.setCacheOperationFacade(mockCacheFacade);
+    balancer.getOOBAuditor().execute();
+
+    assertEquals(0, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
+  }
+  
+  @Test
+  public void testAutoBalanceStatUpdate() {
+    assertEquals(0, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
+    new GeodeCacheFacade().incrementAttemptCounter();
+    
+    assertEquals(1, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
+  }
+  
+  @Test
+  public void testLockSuccess() throws InterruptedException {
+    final AtomicBoolean acquiredAutoBalanceLock = new AtomicBoolean(true);
+    
+    Thread thread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        CacheOperationFacade cacheFacade = new GeodeCacheFacade();
+        acquiredAutoBalanceLock.set(cacheFacade.acquireAutoBalanceLock());
+      }
+    });
+    thread.start();
+    
+    await().atMost(TIMEOUT_SECONDS, SECONDS).untilTrue(acquiredAutoBalanceLock);
+    
+    DistributedLockService dls = new GeodeCacheFacade().getDLS();
+    assertFalse(dls.lock(AutoBalancer.AUTO_BALANCER_LOCK, 0, -1));
+  }
+
+  @Test
+  public void testLockAlreadyTakenElsewhere() throws InterruptedException {
+    DistributedLockService dls = new GeodeCacheFacade().getDLS();
+    assertTrue(dls.lock(AutoBalancer.AUTO_BALANCER_LOCK, 0, -1));
+
+    final AtomicBoolean success = new AtomicBoolean(true);
+    
+    Thread thread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        CacheOperationFacade cacheFacade = new GeodeCacheFacade();
+        success.set(cacheFacade.acquireAutoBalanceLock());
+      }
+    });
+    thread.start();
+    thread.join();
+    
+    assertFalse(success.get());
+  }
+
+  @Test
+  public void testInitializerCacheXML() {
+    String configStr = "<cache xmlns=\"http://schema.pivotal.io/gemfire/cache\"                          "
+        + " xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"                                      "
+        + " xsi:schemaLocation=\"http://schema.pivotal.io/gemfire/cache http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd\""
+        + " version=\"9.0\">                                                                             "
+        + "   <initializer>                                                                              "
+        + "     <class-name>com.gemstone.gemfire.cache.util.AutoBalancer</class-name>                    "
+        + "     <parameter name=\"schedule\">                                                            "
+        + "       <string>* * * * * ? </string>                                                          "
+        + "     </parameter>                                                                             "
+        + "   </initializer>                                                                             "
+        + " </cache>";
+
+    cache.loadCacheXml(new ByteArrayInputStream(configStr.getBytes()));
+  }
+
+  @Test(expected = GemFireConfigException.class)
+  public void testInitFailOnMissingScheduleConf() {
+    String configStr = "<cache xmlns=\"http://schema.pivotal.io/gemfire/cache\"                          "
+        + " xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"                                      "
+        + " xsi:schemaLocation=\"http://schema.pivotal.io/gemfire/cache http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd\""
+        + " version=\"9.0\">                                                                             "
+        + "   <initializer>                                                                              "
+        + "     <class-name>com.gemstone.gemfire.cache.util.AutoBalancer</class-name>                    "
+        + "   </initializer>                                                                             "
+        + " </cache>";
+
+    cache.loadCacheXml(new ByteArrayInputStream(configStr.getBytes()));
+  }
+
+  @Test
+  public void testFacadeCollectMemberDetails2Regions() {
+    final GemFireCacheImpl mockCache = mockContext.mock(GemFireCacheImpl.class);
+
+    final PartitionedRegion mockR1 = mockContext.mock(PartitionedRegion.class, "r1");
+    final PartitionedRegion mockR2 = mockContext.mock(PartitionedRegion.class, "r2");
+    final HashSet<PartitionedRegion> regions = new HashSet<>();
+    regions.add(mockR1);
+    regions.add(mockR2);
+
+    final PRHARedundancyProvider mockRedundancyProviderR1 = mockContext.mock(PRHARedundancyProvider.class, "prhaR1");
+    final InternalPRInfo mockR1PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR1");
+
+    final PRHARedundancyProvider mockRedundancyProviderR2 = mockContext.mock(PRHARedundancyProvider.class, "prhaR2");
+    final InternalPRInfo mockR2PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR2");
+
+    mockContext.checking(new Expectations() {
+      {
+        oneOf(mockCache).getPartitionedRegions();
+        will(returnValue(regions));
+        exactly(2).of(mockCache).getResourceManager();
+        will(returnValue(cache.getResourceManager()));
+        allowing(mockR1).getFullPath();
+        oneOf(mockR1).getRedundancyProvider();
+        will(returnValue(mockRedundancyProviderR1));
+        allowing(mockR2).getFullPath();
+        oneOf(mockR2).getRedundancyProvider();
+        will(returnValue(mockRedundancyProviderR2));
+
+        oneOf(mockRedundancyProviderR1).buildPartitionedRegionInfo(with(true), with(any(LoadProbe.class)));
+        will(returnValue(mockR1PRInfo));
+
+        oneOf(mockRedundancyProviderR2).buildPartitionedRegionInfo(with(true), with(any(LoadProbe.class)));
+        will(returnValue(mockR2PRInfo));
+      }
+    });
+
+    GeodeCacheFacade facade = new GeodeCacheFacade() {
+      @Override
+      GemFireCacheImpl getCache() {
+        return mockCache;
+      }
+    };
+
+    Map<PartitionedRegion, InternalPRInfo> map = facade.getRegionMemberDetails();
+    assertNotNull(map);
+    assertEquals(2, map.size());
+    assertEquals(map.get(mockR1), mockR1PRInfo);
+    assertEquals(map.get(mockR2), mockR2PRInfo);
+  }
+
+  private GemFireCacheImpl createBasicCache() {
+    return (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").create();
+  }
+
+  private Callable<Boolean> isAlive(final HostStatSampler statSampler) {
+    return new Callable<Boolean>() {
+      @Override
+      public Boolean call() throws Exception {
+        return statSampler.isAlive();
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f801d1cf/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
index f0bcded..1eca3c2 100644
--- a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
+++ b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
@@ -1,25 +1,15 @@
 package com.gemstone.gemfire.cache.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
-import java.io.ByteArrayInputStream;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Properties;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.jmock.Expectations;
 import org.jmock.Mockery;
 import org.jmock.Sequence;
-import org.jmock.api.Invocation;
-import org.jmock.lib.action.CustomAction;
 import org.jmock.lib.concurrent.Synchroniser;
 import org.jmock.lib.legacy.ClassImposteriser;
 import org.junit.After;
@@ -28,7 +18,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.GemFireConfigException;
-import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.control.RebalanceFactory;
 import com.gemstone.gemfire.cache.control.RebalanceOperation;
 import com.gemstone.gemfire.cache.control.RebalanceResults;
@@ -38,20 +27,17 @@ import com.gemstone.gemfire.cache.util.AutoBalancer.CacheOperationFacade;
 import com.gemstone.gemfire.cache.util.AutoBalancer.GeodeCacheFacade;
 import com.gemstone.gemfire.cache.util.AutoBalancer.OOBAuditor;
 import com.gemstone.gemfire.cache.util.AutoBalancer.SizeBasedOOBAuditor;
-import com.gemstone.gemfire.cache.util.AutoBalancer.TimeProvider;
-import com.gemstone.gemfire.distributed.DistributedLockService;
-import com.gemstone.gemfire.distributed.internal.locks.DLockService;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.PRHARedundancyProvider;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.control.InternalResourceManager;
 import com.gemstone.gemfire.internal.cache.partitioned.InternalPRInfo;
-import com.gemstone.gemfire.internal.cache.partitioned.LoadProbe;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
+/**
+ * UnitTests for AutoBalancer. All collaborators should be mocked.
+ */
 @Category(UnitTest.class)
 public class AutoBalancerJUnitTest {
-  GemFireCacheImpl cache;
   Mockery mockContext;
 
   @Before
@@ -65,129 +51,13 @@ public class AutoBalancerJUnitTest {
   }
 
   @After
-  public void destroyCacheAndDLS() {
-    if (DLockService.getServiceNamed(AutoBalancer.AUTO_BALANCER_LOCK_SERVICE_NAME) != null) {
-      DLockService.destroy(AutoBalancer.AUTO_BALANCER_LOCK_SERVICE_NAME);
-    }
-
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache = null;
-    }
-  }
-
-  @After
   public void validateMock() {
     mockContext.assertIsSatisfied();
     mockContext = null;
   }
 
-  @Test(expected = IllegalStateException.class)
-  public void testNoCacheError() {
-    AutoBalancer balancer = new AutoBalancer();
-    OOBAuditor auditor = balancer.getOOBAuditor();
-    auditor.execute();
-  }
-
-  @Test
-  public void testAutoRebalaceStatsOnLockSuccess() throws InterruptedException {
-    cache = createBasicCache();
-
-    final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
-    mockContext.checking(new Expectations() {
-      {
-        oneOf(mockCacheFacade).acquireAutoBalanceLock();
-        will(returnValue(true));
-        oneOf(mockCacheFacade).incrementAttemptCounter();
-        will(new CustomAction("increment stat") {
-          public Object invoke(Invocation invocation) throws Throwable {
-            new GeodeCacheFacade().incrementAttemptCounter();
-            return null;
-          }
-        });
-        allowing(mockCacheFacade);
-      }
-    });
-
-    assertEquals(0, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
-    AutoBalancer balancer = new AutoBalancer();
-    balancer.setCacheOperationFacade(mockCacheFacade);
-    balancer.getOOBAuditor().execute();
-    assertEquals(1, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
-  }
-
-  @Test
-  public void testAutoRebalaceStatsOnLockFailure() throws InterruptedException {
-    cache = createBasicCache();
-
-    final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
-    mockContext.checking(new Expectations() {
-      {
-        oneOf(mockCacheFacade).acquireAutoBalanceLock();
-        will(returnValue(false));
-      }
-    });
-
-    assertEquals(0, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
-    AutoBalancer balancer = new AutoBalancer();
-    balancer.setCacheOperationFacade(mockCacheFacade);
-    balancer.getOOBAuditor().execute();
-    assertEquals(0, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
-  }
-
-  @Test
-  public void testAutoBalanceStatUpdate() {
-    cache = createBasicCache();
-    assertEquals(0, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
-    new GeodeCacheFacade().incrementAttemptCounter();
-    assertEquals(1, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
-  }
-
-  @Test
-  public void testLockSuccess() throws InterruptedException {
-    cache = createBasicCache();
-
-    final CountDownLatch locked = new CountDownLatch(1);
-    final AtomicBoolean success = new AtomicBoolean(true);
-    Thread thread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        CacheOperationFacade cacheFacade = new GeodeCacheFacade();
-        success.set(cacheFacade.acquireAutoBalanceLock());
-        locked.countDown();
-      }
-    });
-    thread.start();
-    locked.await(1, TimeUnit.SECONDS);
-    assertTrue(success.get());
-    DistributedLockService dls = new GeodeCacheFacade().getDLS();
-    assertFalse(dls.lock(AutoBalancer.AUTO_BALANCER_LOCK, 0, -1));
-  }
-
-  @Test
-  public void testLockAlreadyTakenElsewhere() throws InterruptedException {
-    cache = createBasicCache();
-
-    DistributedLockService dls = new GeodeCacheFacade().getDLS();
-    assertTrue(dls.lock(AutoBalancer.AUTO_BALANCER_LOCK, 0, -1));
-
-    final AtomicBoolean success = new AtomicBoolean(true);
-    Thread thread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        CacheOperationFacade cacheFacade = new GeodeCacheFacade();
-        success.set(cacheFacade.acquireAutoBalanceLock());
-      }
-    });
-    thread.start();
-    thread.join();
-    assertFalse(success.get());
-  }
-
   @Test
   public void testLockStatExecuteInSequence() throws InterruptedException {
-    cache = createBasicCache();
-
     final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
     final Sequence sequence = mockContext.sequence("sequence");
     mockContext.checking(new Expectations() {
@@ -210,8 +80,6 @@ public class AutoBalancerJUnitTest {
 
   @Test
   public void testReusePreAcquiredLock() throws InterruptedException {
-    cache = createBasicCache();
-
     final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
     mockContext.checking(new Expectations() {
       {
@@ -231,8 +99,6 @@ public class AutoBalancerJUnitTest {
 
   @Test
   public void testAcquireLockAfterReleasedRemotely() throws InterruptedException {
-    cache = createBasicCache();
-
     final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
     final Sequence sequence = mockContext.sequence("sequence");
     mockContext.checking(new Expectations() {
@@ -257,8 +123,6 @@ public class AutoBalancerJUnitTest {
 
   @Test
   public void testFailExecuteIfLockedElsewhere() throws InterruptedException {
-    cache = createBasicCache();
-
     final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
     mockContext.checking(new Expectations() {
       {
@@ -282,8 +146,6 @@ public class AutoBalancerJUnitTest {
 
   @Test
   public void testFailExecuteIfBalanced() throws InterruptedException {
-    cache = createBasicCache();
-
     final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
     mockContext.checking(new Expectations() {
       {
@@ -307,6 +169,13 @@ public class AutoBalancerJUnitTest {
     balancer.getOOBAuditor().execute();
   }
 
+  @Test(expected = IllegalStateException.class)
+  public void testNoCacheError() {
+    AutoBalancer balancer = new AutoBalancer();
+    OOBAuditor auditor = balancer.getOOBAuditor();
+    auditor.execute();
+  }
+
   @Test
   public void testOOBWhenBelowSizeThreshold() {
     final long totalSize = 1000L;
@@ -419,71 +288,6 @@ public class AutoBalancerJUnitTest {
     assertTrue(auditor.needsRebalancing());
   }
 
-  @Test
-  public void testInitializerCacheXML() {
-    String configStr = "<cache xmlns=\"http://schema.pivotal.io/gemfire/cache\"                          "
-        + " xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"                                      "
-        + " xsi:schemaLocation=\"http://schema.pivotal.io/gemfire/cache http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd\""
-        + " version=\"9.0\">                                                                             "
-        + "   <initializer>                                                                              "
-        + "     <class-name>com.gemstone.gemfire.cache.util.AutoBalancer</class-name>                    "
-        + "     <parameter name=\"schedule\">                                                            "
-        + "       <string>* * * * * ? </string>                                                          "
-        + "     </parameter>                                                                             "
-        + "   </initializer>                                                                             "
-        + " </cache>";
-
-    cache = createBasicCache();
-    cache.loadCacheXml(new ByteArrayInputStream(configStr.getBytes()));
-  }
-
-  private GemFireCacheImpl createBasicCache() {
-    return (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").create();
-  }
-
-  @Test(expected = GemFireConfigException.class)
-  public void testInitFailOnMissingScheduleConf() {
-    String configStr = "<cache xmlns=\"http://schema.pivotal.io/gemfire/cache\"                          "
-        + " xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"                                      "
-        + " xsi:schemaLocation=\"http://schema.pivotal.io/gemfire/cache http://schema.pivotal.io/gemfire/cache/cache-9.0.xsd\""
-        + " version=\"9.0\">                                                                             "
-        + "   <initializer>                                                                              "
-        + "     <class-name>com.gemstone.gemfire.cache.util.AutoBalancer</class-name>                    "
-        + "   </initializer>                                                                             "
-        + " </cache>";
-
-    cache = createBasicCache();
-    cache.loadCacheXml(new ByteArrayInputStream(configStr.getBytes()));
-  }
-
-  @Test
-  public void testAuditorInvocation() throws InterruptedException {
-    int count = 0;
-
-    final OOBAuditor mockAuditor = mockContext.mock(OOBAuditor.class);
-    final TimeProvider mockClock = mockContext.mock(TimeProvider.class);
-    mockContext.checking(new Expectations() {
-      {
-        oneOf(mockAuditor).init(with(any(Properties.class)));
-        exactly(2).of(mockAuditor).execute();
-        allowing(mockClock).currentTimeMillis();
-        will(returnValue(950L));
-      }
-    });
-
-    Properties props = getBasicConfig();
-
-    assertEquals(0, count);
-    AutoBalancer autoR = new AutoBalancer();
-    autoR.setOOBAuditor(mockAuditor);
-    autoR.setTimeProvider(mockClock);
-
-    // the trigger should get invoked after 50 milliseconds
-    autoR.init(props);
-
-    TimeUnit.MILLISECONDS.sleep(120);
-  }
-
   @Test(expected = GemFireConfigException.class)
   public void testInvalidSchedule() {
     String someSchedule = "X Y * * * *";
@@ -658,59 +462,6 @@ public class AutoBalancerJUnitTest {
   }
 
   @Test
-  public void testFacadeCollectMemberDetails2Regions() {
-    cache = createBasicCache();
-
-    final GemFireCacheImpl mockCache = mockContext.mock(GemFireCacheImpl.class);
-
-    final PartitionedRegion mockR1 = mockContext.mock(PartitionedRegion.class, "r1");
-    final PartitionedRegion mockR2 = mockContext.mock(PartitionedRegion.class, "r2");
-    final HashSet<PartitionedRegion> regions = new HashSet<>();
-    regions.add(mockR1);
-    regions.add(mockR2);
-
-    final PRHARedundancyProvider mockRedundancyProviderR1 = mockContext.mock(PRHARedundancyProvider.class, "prhaR1");
-    final InternalPRInfo mockR1PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR1");
-
-    final PRHARedundancyProvider mockRedundancyProviderR2 = mockContext.mock(PRHARedundancyProvider.class, "prhaR2");
-    final InternalPRInfo mockR2PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR2");
-
-    mockContext.checking(new Expectations() {
-      {
-        oneOf(mockCache).getPartitionedRegions();
-        will(returnValue(regions));
-        exactly(2).of(mockCache).getResourceManager();
-        will(returnValue(cache.getResourceManager()));
-        allowing(mockR1).getFullPath();
-        oneOf(mockR1).getRedundancyProvider();
-        will(returnValue(mockRedundancyProviderR1));
-        allowing(mockR2).getFullPath();
-        oneOf(mockR2).getRedundancyProvider();
-        will(returnValue(mockRedundancyProviderR2));
-
-        oneOf(mockRedundancyProviderR1).buildPartitionedRegionInfo(with(true), with(any(LoadProbe.class)));
-        will(returnValue(mockR1PRInfo));
-
-        oneOf(mockRedundancyProviderR2).buildPartitionedRegionInfo(with(true), with(any(LoadProbe.class)));
-        will(returnValue(mockR2PRInfo));
-      }
-    });
-
-    GeodeCacheFacade facade = new GeodeCacheFacade() {
-      @Override
-      GemFireCacheImpl getCache() {
-        return mockCache;
-      }
-    };
-
-    Map<PartitionedRegion, InternalPRInfo> map = facade.getRegionMemberDetails();
-    assertNotNull(map);
-    assertEquals(2, map.size());
-    assertEquals(map.get(mockR1), mockR1PRInfo);
-    assertEquals(map.get(mockR2), mockR2PRInfo);
-  }
-
-  @Test
   public void testFacadeTotalBytes2Regions() {
     final PartitionedRegion mockR1 = mockContext.mock(PartitionedRegion.class, "r1");
     final PartitionedRegion mockR2 = mockContext.mock(PartitionedRegion.class, "r2");
@@ -763,7 +514,7 @@ public class AutoBalancerJUnitTest {
     assertEquals(123 + 74 + 3475, facade.getTotalDataSize(details));
   }
 
-  private Properties getBasicConfig() {
+  static Properties getBasicConfig() {
     Properties props = new Properties();
     // every second schedule
     props.put(AutoBalancer.SCHEDULE, "* * * * * ?");