You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2016/04/21 19:16:58 UTC

[23/50] [abbrv] incubator-geode git commit: GEODE-1233: apply FlakyTest category to flickering tests

GEODE-1233: apply FlakyTest category to flickering tests

* perform basic analysis on all JIRA tickets filed against flickering (intermittently failing) tests
* annotate each flickering test method with "@Category(FlakyTest.class) // GEODE-xxx: basic analysis of pausible causes for flakiness"
* label each flickering JIRA ticket with "Flaky"
* add comment to each flickering JIRA ticket with the same "basic analysis of pausible causes for flakiness"
* fix some obvious cases of "eats exceptions"
* add some TODOs to problem areas found in code


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

Branch: refs/heads/feature/GEODE-17-2
Commit: f12ece597eb003c18e752328526cd75df1609ae1
Parents: 1fa1571
Author: Kirk Lund <kl...@apache.org>
Authored: Wed Apr 20 09:36:54 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed Apr 20 09:38:39 2016 -0700

----------------------------------------------------------------------
 .../SessionReplicationIntegrationJUnitTest.java |  10 +-
 .../gemfire/internal/AvailablePort.java         |   1 -
 .../com/gemstone/gemfire/TXExpiryJUnitTest.java |   6 +-
 .../gemfire/cache/ConnectionPoolDUnitTest.java  |   3 +
 .../pooling/ConnectionManagerJUnitTest.java     |   8 +-
 .../management/MemoryThresholdsDUnitTest.java   |   4 +
 .../MemoryThresholdsOffHeapDUnitTest.java       |   9 +-
 .../management/ResourceManagerDUnitTest.java    |  16 +-
 .../dunit/QueryDataInconsistencyDUnitTest.java  |  12 +-
 .../QueryUsingFunctionContextDUnitTest.java     |  11 +-
 .../query/dunit/QueryUsingPoolDUnitTest.java    |   3 +
 .../cache/query/dunit/RemoteQueryDUnitTest.java |   3 +
 .../QueryREUpdateInProgressJUnitTest.java       |   3 +-
 ...pdateWithInplaceObjectModFalseDUnitTest.java |   9 +-
 ...itializeIndexEntryDestroyQueryDUnitTest.java |  14 +-
 .../partitioned/PRQueryCacheCloseDUnitTest.java |  29 +--
 .../gemfire/cache30/Bug44418JUnitTest.java      |  10 +-
 .../gemfire/cache30/CacheLogRollDUnitTest.java  |  13 +-
 .../cache30/ClientMembershipDUnitTest.java      |   4 +
 ...iskDistributedNoAckAsyncRegionDUnitTest.java |   4 -
 .../DiskDistributedNoAckRegionTestCase.java     |   7 +-
 ...tributedAckPersistentRegionCCEDUnitTest.java |  80 ++++----
 .../DistributedAckRegionCCEDUnitTest.java       |  70 +++----
 .../cache30/DistributedAckRegionDUnitTest.java  |   2 -
 .../DistributedNoAckRegionCCEDUnitTest.java     |  76 +++----
 .../DistributedNoAckRegionDUnitTest.java        |   3 +-
 .../gemfire/cache30/MultiVMRegionTestCase.java  |  18 +-
 .../cache30/PartitionedRegionDUnitTest.java     |   6 +-
 .../cache30/RegionReliabilityTestCase.java      |   4 +
 .../gemfire/cache30/RegionTestCase.java         |  11 +-
 .../gemfire/cache30/TXDistributedDUnitTest.java |   5 +-
 .../gemfire/distributed/LocatorDUnitTest.java   |  11 +-
 .../LocatorLauncherRemoteJUnitTest.java         |  13 +-
 .../ServerLauncherRemoteJUnitTest.java          |  11 +-
 .../internal/LocatorLoadSnapshotJUnitTest.java  |   2 +
 .../GemFireDeadlockDetectorDUnitTest.java       |  12 +-
 .../gemfire/disttx/DistTXExpiryJUnitTest.java   |   1 -
 ...hreadPoolExecutorWithKeepAliveJUnitTest.java |   9 +-
 .../cache/FixedPRSinglehopDUnitTest.java        |   4 +
 .../internal/cache/GIIDeltaDUnitTest.java       |   9 +-
 .../cache/NetSearchMessagingDUnitTest.java      |  12 +-
 .../gemfire/internal/cache/OplogJUnitTest.java  |  13 +-
 .../PartitionedRegionCreationDUnitTest.java     |  17 +-
 ...rtitionedRegionDelayedRecoveryDUnitTest.java |  13 +-
 .../PartitionedRegionSingleHopDUnitTest.java    |   5 +
 .../internal/cache/SingleHopStatsDUnitTest.java |   9 +-
 ...ributedRegionFunctionExecutionDUnitTest.java |  11 +-
 ...onFunctionExecutionNoSingleHopDUnitTest.java |   7 +-
 .../PRFunctionExecutionTimeOutDUnitTest.java    |  17 +-
 .../internal/cache/ha/Bug48571DUnitTest.java    |   6 +-
 .../cache/ha/HARegionQueueDUnitTest.java        |  14 +-
 ...tentColocatedPartitionedRegionDUnitTest.java |  16 +-
 .../PersistentPartitionedRegionDUnitTest.java   |  15 +-
 ...tentPartitionedRegionOldConfigDUnitTest.java |   6 +-
 .../fixed/FixedPartitioningDUnitTest.java       |  12 +-
 .../DestroyEntryPropagationDUnitTest.java       |  14 +-
 .../sockets/HAStartupAndFailoverDUnitTest.java  |  21 +-
 .../sockets/ReliableMessagingDUnitTest.java     |  12 +-
 .../sockets/UpdatePropagationDUnitTest.java     |  17 +-
 .../asyncqueue/AsyncEventListenerDUnitTest.java |   3 +
 .../AsyncEventListenerOffHeapDUnitTest.java     |   4 +-
 .../management/DLockManagementDUnitTest.java    |  22 +-
 ...ersalMembershipListenerAdapterDUnitTest.java |   5 +-
 .../cli/commands/CliCommandTestBase.java        |  14 +-
 ...eateAlterDestroyRegionCommandsDUnitTest.java |  10 +-
 .../commands/DiskStoreCommandsDUnitTest.java    |   2 +
 .../commands/GemfireDataCommandsDUnitTest.java  |   2 +
 .../cli/commands/IndexCommandsDUnitTest.java    |   4 +-
 .../ListAndDescribeRegionDUnitTest.java         |   2 +
 .../MiscellaneousCommandsDUnitTest.java         |   3 +-
 ...laneousCommandsExportLogsPart3DUnitTest.java |   2 +
 .../cli/commands/ShellCommandsDUnitTest.java    |   2 +
 .../SharedConfigurationUsingDirDUnitTest.java   |   2 +
 .../GemcachedDevelopmentJUnitTest.java          |   2 +
 .../pdx/DistributedSystemIdDUnitTest.java       |  13 +-
 .../gemfire/redis/RedisDistDUnitTest.java       |   4 +
 .../security/ClientAuthenticationDUnitTest.java |   2 +
 .../security/P2PAuthenticationDUnitTest.java    |   3 +
 .../cache/query/cq/dunit/CqPerfDUnitTest.java   |   5 +-
 .../cq/dunit/CqPerfUsingPoolDUnitTest.java      |   9 +-
 .../cache/query/cq/dunit/CqQueryDUnitTest.java  |   2 +-
 .../cq/dunit/CqQueryUsingPoolDUnitTest.java     |  22 +-
 .../query/cq/dunit/PrCqUsingPoolDUnitTest.java  |  18 +-
 .../query/dunit/QueryMonitorDUnitTest.java      |   8 +-
 .../sockets/DurableClientSimpleDUnitTest.java   | 203 ++++++++++---------
 .../cli/commands/ClientCommandsDUnitTest.java   |   7 +-
 .../security/ClientAuthzObjectModDUnitTest.java |   4 +-
 .../ClientPostAuthorizationDUnitTest.java       |   9 +-
 .../LuceneIndexRecoveryHAJUnitTest.java         |   5 +-
 .../cache/CacheXml70GatewayDUnitTest.java       |  18 +-
 .../cache/wan/CacheClientNotifierDUnitTest.java |   4 +
 .../gemfire/internal/cache/wan/WANTestBase.java |  13 +-
 ...allelGatewaySenderOperation_1_DUnitTest.java |   7 +-
 .../ConcurrentWANPropogation_2_DUnitTest.java   |  10 +-
 .../cache/wan/misc/WANSSLDUnitTest.java         |   4 +
 .../cache/wan/misc/WanValidationsDUnitTest.java |   8 +-
 .../ParallelWANPropogationOffHeapDUnitTest.java |   3 +-
 .../SerialWANPropogationOffHeapDUnitTest.java   |   3 +-
 ...ation_PartitionedRegionOffHeapDUnitTest.java |   3 +-
 ...arallelGatewaySenderOperationsDUnitTest.java |   6 +-
 ...ersistenceEnabledGatewaySenderDUnitTest.java |   7 +-
 .../ParallelWANPropagationDUnitTest.java        |   4 +
 .../wan/parallel/ParallelWANStatsDUnitTest.java |   5 +
 ...rialGatewaySenderEventListenerDUnitTest.java |  10 +-
 .../SerialWANPropagationLoopBackDUnitTest.java  |   6 +-
 .../serial/SerialWANPropogationDUnitTest.java   |   9 +-
 ...NPropogation_PartitionedRegionDUnitTest.java |   8 +-
 107 files changed, 708 insertions(+), 576 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java b/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
index 0abd85e..004f9fe 100644
--- a/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
+++ b/extensions/geode-modules-session/src/test/java/com/gemstone/gemfire/modules/session/internal/filter/SessionReplicationIntegrationJUnitTest.java
@@ -14,7 +14,6 @@
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
-
 package com.gemstone.gemfire.modules.session.internal.filter;
 
 import java.io.File;
@@ -37,6 +36,7 @@ import javax.servlet.http.HttpSession;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.modules.session.junit.PerTestClassLoaderRunner;
 import com.gemstone.gemfire.modules.session.filter.SessionCachingFilter;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.apache.jasper.servlet.JspServlet;
 import org.junit.After;
@@ -74,7 +74,7 @@ public class SessionReplicationIntegrationJUnitTest {
 
   static {
     // Create a per-user scratch directory
-    tmpdir = new File(System.getProperty("java.io.tmpdir"),
+    tmpdir = new File(System.getProperty("java.io.tmpdir"), // TODO: use junit rule TemporaryFolder
         "gemfire_modules-" + System.getProperty("user.name"));
     tmpdir.mkdirs();
     tmpdir.deleteOnExit();
@@ -355,7 +355,8 @@ public class SessionReplicationIntegrationJUnitTest {
 
   /**
    * Test setting an attribute to null deletes it
-   */
+  */
+  @Category(FlakyTest.class) // GEODE-1015: uses Jetty HttpTester, uses static vars with class loader isolation, TODO: rewrite test with JUnit 4 rules including TemporaryFolder
   @Test
   public void testSetAttributeNullDeletesIt() throws Exception {
     Callback c_1 = new Callback() {
@@ -485,6 +486,7 @@ public class SessionReplicationIntegrationJUnitTest {
    * Test that invalidating a session destroys it as well as the backend
    * object.
    */
+  @Category(FlakyTest.class) // GEODE-1015: uses Jetty HttpTester, uses static vars with class loader isolation, TODO: rewrite test with JUnit 4 rules including TemporaryFolder
   @Test
   public void testInvalidateSession1() throws Exception {
     Callback c_1 = new Callback() {
@@ -575,6 +577,7 @@ public class SessionReplicationIntegrationJUnitTest {
   /**
    * Test that invalidating a session throws an exception on subsequent access.
    */
+  @Category(FlakyTest.class) // GEODE-1015: uses Jetty HttpTester, uses static vars with class loader isolation, TODO: rewrite test with JUnit 4 rules including TemporaryFolder
   @Test
   public void testInvalidateSession3() throws Exception {
     Callback c_1 = new Callback() {
@@ -1206,6 +1209,7 @@ public class SessionReplicationIntegrationJUnitTest {
   /**
    * Test that request forward dispatching works
    */
+  @Category(FlakyTest.class) // GEODE-1015: uses Jetty HttpTester, uses static vars with class loader isolation, TODO: rewrite test with JUnit 4 rules including TemporaryFolder
   @Test
   public void testDispatchingForward1() throws Exception {
     Callback c_1 = new Callback() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
index c222490..11eb834 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
@@ -28,7 +28,6 @@ import java.util.Random;
 /**
  * This class determines whether or not a given port is available and
  * can also provide a randomly selected available port.
- *
  */
 public class AvailablePort {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
index 937d8e4..30dacd1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
@@ -51,15 +51,16 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.TXManagerImpl;
 import com.gemstone.gemfire.internal.cache.TXStateProxy;
+import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Tests transaction expiration functionality
  *
  * @since 4.0
- *
  */
 @Category(IntegrationTest.class)
 public class TXExpiryJUnitTest {
@@ -326,6 +327,7 @@ public class TXExpiryJUnitTest {
     }
   }
 
+  @Category(FlakyTest.class) // GEODE-845: time sensitive, expiration, eats exceptions (1 fixed), waitForCriterion, 3 second timeout
   @Test
   public void testRegionIdleExpiration() throws CacheException {
     Region<String, String> exprReg = createRegion("TXRegionIdle");
@@ -392,7 +394,7 @@ public class TXExpiryJUnitTest {
         ExpiryTask.suspendExpiration();
         this.txMgr.commit();
       } catch (CommitConflictException error) {
-        fail("Expiration should not cause commit to fail");
+        Assert.fail("Expiration should not cause commit to fail", error);
       }
       assertEquals("value", exprReg.getEntry("key0").getValue());
       waitForRegionExpiration(lr, useTTL);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java
index 2acab3a..bbc2839 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolDUnitTest.java
@@ -31,6 +31,7 @@ import java.util.Set;
 import junit.framework.AssertionFailedError;
 
 import org.junit.FixMethodOrder;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.LogWriter;
@@ -70,6 +71,7 @@ import com.gemstone.gemfire.test.dunit.ThreadUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * This class tests the client connection pool in GemFire.
@@ -1186,6 +1188,7 @@ public class ConnectionPoolDUnitTest extends CacheTestCase {
    * Make sure cnx lifetime expiration working on thread local cnxs.
    * @author darrel
    */
+  @Category(FlakyTest.class) // GEODE-1197: random ports, BindException, FixMethodOrder, expiration, time sensitive, waitForCriterion, async actions
   public void test009LifetimeExpireOnTL() throws CacheException {
     basicTestLifetimeExpire(true);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
index 1250d5a..6360d59 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/client/internal/pooling/ConnectionManagerJUnitTest.java
@@ -62,13 +62,12 @@ import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.test.dunit.ThreadUtils;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-/**
- *
- */
 @Category(IntegrationTest.class)
 public class ConnectionManagerJUnitTest {
+
   private static final long TIMEOUT = 30 * 1000;
   //This is added for some windows machines which think the connection expired
   //before the idle timeout due to precision issues.
@@ -228,7 +227,8 @@ public class ConnectionManagerJUnitTest {
 //    }
 //    
 //  }
-  
+
+  @Category(FlakyTest.class) // GEODE-923: time sensitive, expiration, thread sleeps, wait loop
   @Test
   public void testIdleExpiration() throws InterruptedException, AllConnectionsInUseException, NoAvailableServersException {
     final long nanoToMillis = 1000000;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java
index 8f8c966..c6e0568 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java
@@ -29,6 +29,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.junit.experimental.categories.Category;
 import util.TestException;
 
 import com.gemstone.gemfire.cache.AttributesFactory;
@@ -93,6 +94,7 @@ import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Tests the Heap Memory thresholds of {@link ResourceManager}
@@ -398,6 +400,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
    * 
    * @throws Exception
    */
+  @Category(FlakyTest.class) // GEODE-427: random ports, time sensitive, waitForCriterions
   public void testEventDelivery() throws Exception {
     final Host host = Host.getHost(0);
     final VM server1 = host.getVM(0);
@@ -566,6 +569,7 @@ public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
     prRemotePutRejection(true, false, true);
   }
 
+  @Category(FlakyTest.class) // GEODE-987: random ports, failed to throw expected ResourceException, overly complex expected exception handling, memory and GC sensitive, expiration, waitForCriterion
   public void testPR_RemotePutRejectionWithTx() throws Exception {
     prRemotePutRejection(false, false, true);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
index feefb6d..be6b266 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
@@ -26,6 +26,8 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.AttributesMutator;
 import com.gemstone.gemfire.cache.CacheException;
@@ -82,6 +84,7 @@ import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Tests the Off-Heap Memory thresholds of {@link ResourceManager}
@@ -512,6 +515,7 @@ public class MemoryThresholdsOffHeapDUnitTest extends ClientServerTestCase {
    * to a safe state then test that they are allowed.
    * @throws Exception
    */
+  @Category(FlakyTest.class) // GEODE-438: test pollution, async actions, time sensitive, waitForCriterion, TODO: consider disconnect DS in setup
   public void testDRLoadRejection() throws Exception {
     final Host host = Host.getHost(0);
     final VM replicate1 = host.getVM(1);
@@ -725,6 +729,7 @@ public class MemoryThresholdsOffHeapDUnitTest extends ClientServerTestCase {
     prRemotePutRejection(false, true, false);
   }
 
+  @Category(FlakyTest.class) // GEODE-596: BindException, random ports
   public void testPR_RemotePutRejectionCacheClose() throws Exception {
     prRemotePutRejection(true, false, false);
   }
@@ -741,6 +746,7 @@ public class MemoryThresholdsOffHeapDUnitTest extends ClientServerTestCase {
     prRemotePutRejection(true, false, true);
   }
 
+  @Category(FlakyTest.class) // GEODE-500: random ports, time sensitive, memory sensitive and GC dependent, waitForCriterions
   public void testPR_RemotePutRejectionWithTx() throws Exception {
     prRemotePutRejection(false, false, true);
   }
@@ -916,13 +922,14 @@ public class MemoryThresholdsOffHeapDUnitTest extends ClientServerTestCase {
    * if the VM with the bucket is in a critical state.
    * @throws Exception
    */
+  @Category(FlakyTest.class) // GEODE-551: waitForCriterion, memory sensitive
   public void testPRLoadRejection() throws Exception {
     final Host host = Host.getHost(0);
     final VM accessor = host.getVM(1);
     final VM ds1 = host.getVM(2);
     final String rName = getUniqueName();
 
-    // Make sure the desired VMs will have a fresh DS.
+    // Make sure the desired VMs will have a fresh DS. TODO: convert these from AsyncInvocation to invoke
     AsyncInvocation d0 = accessor.invokeAsync(() -> DistributedTestCase.disconnectFromDS());
     AsyncInvocation d1 = ds1.invokeAsync(() -> DistributedTestCase.disconnectFromDS());
     d0.join();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java
index 8c455d4..9f57fc0 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/management/ResourceManagerDUnitTest.java
@@ -22,6 +22,7 @@ import java.util.Iterator;
 import java.util.Set;
 
 import org.apache.logging.log4j.Logger;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
@@ -66,13 +67,13 @@ import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Tests com.gemstone.gemfire.cache.control.ResourceManager.
  * 
  * TODO: javadoc this test properly and cleanup the helper methods to be
  * more flexible and understandable
- *  
  */
 public class ResourceManagerDUnitTest extends CacheTestCase {
   private static final Logger logger = LogService.getLogger();
@@ -968,6 +969,8 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
         }
       });
   }
+
+  @Category(FlakyTest.class) // GEODE-755: thread unsafe test hook (bucketReadHook), remove bucket fails, possible product bug in rebalancing
   public void testRemoveDuringGet() {
     doOpDuringBucketRemove(new OpDuringBucketRemove() {
         public void runit(PartitionedRegion pr, Object key, Object value) {
@@ -975,6 +978,8 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
         }
       });
   }
+
+  @Category(FlakyTest.class) // GEODE-673: thread unsafe test hook (bucketReadHook), remove bucket fails, possible product bug in rebalancing
   public void testRemoveDuringContainsKey() {
     doOpDuringBucketRemove(new OpDuringBucketRemove() {
         public void runit(PartitionedRegion pr, Object key, Object value) {
@@ -982,6 +987,7 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
         }
       });
   }
+
   public void testRemoveDuringContainsValueForKey() {
     doOpDuringBucketRemove(new OpDuringBucketRemove() {
         public void runit(PartitionedRegion pr, Object key, Object value) {
@@ -989,6 +995,7 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
         }
       });
   }
+
   public void testRemoveDuringKeySet() {
     doOpDuringBucketRemove(new OpDuringBucketRemove() {
         public void runit(PartitionedRegion pr, Object key, Object value) {
@@ -996,6 +1003,7 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
         }
       });
   }
+
   public void testRemoveDuringValues() {
     doOpDuringBucketRemove(new OpDuringBucketRemove() {
         public void runit(PartitionedRegion pr, Object key, Object value) {
@@ -1003,6 +1011,7 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
         }
       });
   }
+
   public void testRemoveDuringEntrySet() {
     doOpDuringBucketRemove(new OpDuringBucketRemove() {
         public void runit(PartitionedRegion pr, Object key, Object value) {
@@ -1014,6 +1023,7 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
         }
       });
   }
+
   public void testRemoveDuringQuery() {
     doOpDuringBucketRemove(new OpDuringBucketRemove() {
         public void runit(PartitionedRegion pr, Object key, Object value) {
@@ -1029,8 +1039,7 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
         }
       });
   }
-  
-  
+
   public void testRemoveBucketMessage() {
     final String[] regionPath = new String[] {
         getUniqueName() + "-PR-0"
@@ -1129,6 +1138,7 @@ public class ResourceManagerDUnitTest extends CacheTestCase {
    * Creates a chain of three colocated PRs and then calls removeBucket
    * to make sure that all colocated buckets are removed together.
    */
+  @Category(FlakyTest.class) // GEODE-928: RemoveBucketMessage failure?
   public void testRemoveColocatedBuckets() {
     final String[] regionPath = new String[] {
         getUniqueName() + "-PR-0", 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryDataInconsistencyDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryDataInconsistencyDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryDataInconsistencyDUnitTest.java
index 5d44dfd..c5f5140 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryDataInconsistencyDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryDataInconsistencyDUnitTest.java
@@ -18,6 +18,8 @@ package com.gemstone.gemfire.cache.query.dunit;
 
 import java.util.Properties;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -49,12 +51,11 @@ import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.ThreadUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * This tests the data inconsistency during update on an index and querying the
  * same UNLOCKED index.
- * 
- * 
  */
 public class QueryDataInconsistencyDUnitTest extends CacheTestCase {
 
@@ -293,8 +294,9 @@ public class QueryDataInconsistencyDUnitTest extends CacheTestCase {
     });
     ThreadUtils.join(putThread, 200);
   }
-  
-  public void testRangeIndexWithIndexAndQueryFromCluaseMisMatch() {
+
+  @Category(FlakyTest.class) // GEODE-925: time sensitive, async actions, short timeouts
+  public void testRangeIndexWithIndexAndQueryFromCluaseMisMatch() { // TODO: fix misspelling
     // Create caches
     Properties props = new Properties();
     server.invoke(() -> PRClientServerTestBase.createCacheInVm( props ));
@@ -381,7 +383,7 @@ public class QueryDataInconsistencyDUnitTest extends CacheTestCase {
         }
       }
     });
-    ThreadUtils.join(putThread, 200);
+    ThreadUtils.join(putThread, 200); // GEODE-925 occurs here and this is very short join 200 millis
   }
 
   public void testRangeIndexWithIndexAndQueryFromCluaseMisMatch2() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingFunctionContextDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingFunctionContextDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingFunctionContextDUnitTest.java
index c85a6d9..08626de 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingFunctionContextDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingFunctionContextDUnitTest.java
@@ -22,6 +22,8 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Set;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -63,13 +65,12 @@ import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * This tests the querying using a RegionFunctionContext which provides a filter
  * (routing keys) to run the query on subset of buckets "locally". If query
  * includes buckets
- *
- *
  */
 public class QueryUsingFunctionContextDUnitTest extends CacheTestCase {
 
@@ -434,10 +435,8 @@ public class QueryUsingFunctionContextDUnitTest extends CacheTestCase {
 
   }
 
-  /**
-  *
-  */
- public void testQueriesWithFilterKeysOnPRWithRebalancing() {
+  @Category(FlakyTest.class) // GEODE-575: ignores lots of exceptions, non-thread-safe test hooks
+  public void testQueriesWithFilterKeysOnPRWithRebalancing() {
    IgnoredException.addIgnoredException("QueryInvocationTargetException");
    IgnoredException.addIgnoredException("java.net.SocketException");
    IgnoredException.addIgnoredException("ServerConnectivityException");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
index 4a4c38d..223e4b4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
@@ -26,6 +26,7 @@ import java.util.Set;
 
 import cacheRunner.Portfolio;
 import cacheRunner.Position;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.DataSerializable;
 import com.gemstone.gemfire.DataSerializer;
@@ -58,6 +59,7 @@ import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Tests remote (client/server) query execution.
@@ -1454,6 +1456,7 @@ public class QueryUsingPoolDUnitTest extends CacheTestCase {
   /**
    * Tests client-server query using parameters (compiled queries).
    */
+  @Category(FlakyTest.class) // GEODE-1146: time senstiive, thread sleeps, uses zero port for servers (good!), async actions, AsyncInvocation orphans
   public void testBindParamsWithMulitipleClients() throws CacheException {
 
     final String name = this.getName();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
index 0853162..6769a0e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
@@ -50,9 +50,11 @@ import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 import cacheRunner.Portfolio;
 import cacheRunner.Position;
+import org.junit.experimental.categories.Category;
 
 /**
  * Tests remote (client/server) query execution.
@@ -939,6 +941,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
    * Tests remote query execution using a BridgeClient as the CacheWriter
    * and CacheLoader.
    */
+  @Category(FlakyTest.class) // GEODE-490: random port
   public void testRemoteBridgeClientQueries() throws CacheException {
 
     final String name = this.getName();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/QueryREUpdateInProgressJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/QueryREUpdateInProgressJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/QueryREUpdateInProgressJUnitTest.java
index 677321f..e7681b5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/QueryREUpdateInProgressJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/functional/QueryREUpdateInProgressJUnitTest.java
@@ -44,6 +44,7 @@ import com.gemstone.gemfire.cache.query.types.ObjectType;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion.NonTXEntry;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
@@ -55,7 +56,7 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
  *
  *
  */
-@Category(IntegrationTest.class)
+@Category({ IntegrationTest.class, FlakyTest.class }) // GEODE-1059: uses PRQueryHelper which launches dunit vms in IntegrationTest
 public class QueryREUpdateInProgressJUnitTest {
 
   private static final String exampleRegionName = "exampleRegion2";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexUpdateWithInplaceObjectModFalseDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexUpdateWithInplaceObjectModFalseDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexUpdateWithInplaceObjectModFalseDUnitTest.java
index 8fc8049..8034931 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexUpdateWithInplaceObjectModFalseDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexUpdateWithInplaceObjectModFalseDUnitTest.java
@@ -21,6 +21,8 @@ package com.gemstone.gemfire.cache.query.internal.index;
 
 import java.util.Collection;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.Region;
@@ -50,6 +52,7 @@ import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
 import com.gemstone.gemfire.test.dunit.ThreadUtils;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * This test is similar to {@link ConcurrentIndexUpdateWithoutWLDUnitTest} except
@@ -58,11 +61,8 @@ import com.gemstone.gemfire.test.dunit.ThreadUtils;
  * 
  * During validation all region operations are paused for a while. Validation
  * happens multiple time during one test run on a fixed time interval.
- * 
- * 
  */
-public class ConcurrentIndexUpdateWithInplaceObjectModFalseDUnitTest extends
-    DistributedTestCase {
+public class ConcurrentIndexUpdateWithInplaceObjectModFalseDUnitTest extends DistributedTestCase {
   
   PRQueryDUnitHelper helper = new PRQueryDUnitHelper("ConcurrentIndexUpdateWithoutWLDUnitTest");
   private static String regionName = "Portfolios";
@@ -173,6 +173,7 @@ public class ConcurrentIndexUpdateWithInplaceObjectModFalseDUnitTest extends
     };
   }
 
+  @Category(FlakyTest.class) // GEODE-431: time sensitive, uses PRQueryDUnitHelper whcih eats exceptions, async actions, uses Random
   public void testRangeIndex() {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/InitializeIndexEntryDestroyQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/InitializeIndexEntryDestroyQueryDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/InitializeIndexEntryDestroyQueryDUnitTest.java
index 6463e88..0311f38 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/InitializeIndexEntryDestroyQueryDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/InitializeIndexEntryDestroyQueryDUnitTest.java
@@ -18,6 +18,8 @@ package com.gemstone.gemfire.cache.query.internal.index;
 
 import java.util.Arrays;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
@@ -41,13 +43,12 @@ import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.ThreadUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Test creates a local region. Creates and removes index in a parallel running thread.
  * Then destroys and puts back entries in separated thread in the same region and runs
  * query parallely and checks for UNDEFINED values in result set of the query.
- *
- *
  */
 public class InitializeIndexEntryDestroyQueryDUnitTest extends CacheTestCase {
 
@@ -207,6 +208,7 @@ public class InitializeIndexEntryDestroyQueryDUnitTest extends CacheTestCase {
     }
   }
 
+  @Category(FlakyTest.class) // GEODE-1036: uses PRQueryDUnitHelper, time sensitive, async actions, overly long joins (16+ minutes), eats exceptions (fixed 1), thread sleeps
   public void testAsyncIndexInitDuringEntryDestroyAndQueryOnPR() {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
@@ -255,7 +257,7 @@ public class InitializeIndexEntryDestroyQueryDUnitTest extends CacheTestCase {
             index = cache.getQueryService().createIndex("statusIndex", "p.status", "/"+name+" p");
           } catch (Exception e1) {
             e1.printStackTrace();
-            fail("Index creation failed");
+            Assert.fail("Index creation failed", e1);
           }
           assertNotNull(index);
 
@@ -320,7 +322,7 @@ public class InitializeIndexEntryDestroyQueryDUnitTest extends CacheTestCase {
             PRQHelp.getCache().getLogger().fine("Querying the region");
             results = (SelectResults)query.execute();
           } catch (Exception e) {
-            e.printStackTrace();
+            e.printStackTrace(); // TODO: eats exceptions
           }
 
           for (Object obj : results) {
@@ -332,12 +334,12 @@ public class InitializeIndexEntryDestroyQueryDUnitTest extends CacheTestCase {
       }
     });
 
-    ThreadUtils.join(asyInvk0, 1000 * 1000);
+    ThreadUtils.join(asyInvk0, 1000 * 1000); // TODO: this is way too long: 16.67 minutes!
     if (asyInvk0.exceptionOccurred()) {
       Assert.fail("asyInvk0 failed", asyInvk0.getException());
     }
     
-    ThreadUtils.join(asyInvk1, 1000 * 1000);
+    ThreadUtils.join(asyInvk1, 1000 * 1000); // TODO: this is way too long: 16.67 minutes!
     if (asyInvk1.exceptionOccurred()) {
       Assert.fail("asyInvk1 failed", asyInvk1.getException());
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryCacheCloseDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryCacheCloseDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryCacheCloseDUnitTest.java
index 60fd6ff..af53e11 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryCacheCloseDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/partitioned/PRQueryCacheCloseDUnitTest.java
@@ -14,22 +14,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.cache.query.partitioned;
 
-/**
- * This test tests the PR query behaviour with respect to cache closure
- * happening on one of the data stores. PR is configured with redundantCopies =
- * 1, and cache close is done randomly on one of the data stores and then
- * recreated, thus avoiding any data loss.
- * 
- */
-
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.query.data.PortfolioData;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDUnitTestCase;
@@ -40,18 +33,17 @@ import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.ThreadUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
-public class PRQueryCacheCloseDUnitTest extends PartitionedRegionDUnitTestCase
-{
-
-  /**
-   * constructor *
-   * 
-   * @param name
-   */
+/**
+ * This test tests the PR query behaviour with respect to cache closure
+ * happening on one of the data stores. PR is configured with redundantCopies =
+ * 1, and cache close is done randomly on one of the data stores and then
+ * recreated, thus avoiding any data loss.
+ */
+public class PRQueryCacheCloseDUnitTest extends PartitionedRegionDUnitTestCase {
 
   public PRQueryCacheCloseDUnitTest(String name) {
-
     super(name);
   }
 
@@ -216,6 +208,7 @@ public class PRQueryCacheCloseDUnitTest extends PartitionedRegionDUnitTestCase
    * 6. then recreates the PR on the same VM <br>
    * 7. Verfies the size , type , contents of both the resultSets Obtained <br>
    */
+  @Category(FlakyTest.class) // GEODE-1239: uses PRQueryDUnitHelper, uses Random, async actions, time sensitive, complex retry loop, thread unsafe test hook
   public void testPRWithCacheCloseInOneDatastoreWithoutDelay() throws Exception
   {
     LogWriterUtils.getLogWriter()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug44418JUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug44418JUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug44418JUnitTest.java
index ccd9df0..a34b9af 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug44418JUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug44418JUnitTest.java
@@ -37,21 +37,22 @@ import com.gemstone.gemfire.cache.Region.Entry;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-
 /**
  * Test for Bug 44418.
  * 
  * @since 7.0
  */
 @Category(IntegrationTest.class)
-public class Bug44418JUnitTest {
+@SuppressWarnings({ "unchecked", "rawtypes" })
+public class Bug44418JUnitTest { // TODO: rename this test to non-ticket descriptive name
 
   DistributedSystem ds;
   Cache cache;
 
-  @SuppressWarnings({ "unchecked", "rawtypes" })
+  @Category(FlakyTest.class) // GEODE-1139: time sensitive, thread sleep, expiration
   @Test
   public void testPut() throws Exception {
 
@@ -98,7 +99,8 @@ public class Bug44418JUnitTest {
       System.getProperties().remove(LocalRegion.EXPIRY_MS_PROPERTY);
     }
   }
-  @SuppressWarnings({ "unchecked", "rawtypes" })
+
+  @Category(FlakyTest.class) // GEODE-924: expiration, time sensitive, expects action in 1 second
   @Test
   public void testGet() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLogRollDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLogRollDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLogRollDUnitTest.java
index 3e5994d..3a96728 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLogRollDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheLogRollDUnitTest.java
@@ -25,9 +25,12 @@ import java.io.IOException;
 import java.util.Properties;
 import java.util.regex.Pattern;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Test to make sure cache close is working.
@@ -233,7 +236,8 @@ public class CacheLogRollDUnitTest extends CacheTestCase {
       * This was throwing NPEs until my fix...
       */
   }
- 
+
+  @Category(FlakyTest.class) // GEODE-674: possible disk pollution, file size sensitive
   public void testSimpleStartRestartWithRolling() throws Exception {
     Properties props = new Properties();
     String baseLogName = "restarto";
@@ -292,7 +296,8 @@ public class CacheLogRollDUnitTest extends CacheTestCase {
     }
 
   }
-  
+
+  @Category(FlakyTest.class) // GEODE-677: possible disk pollution, file size sensitive
   public void testStartWithRollingThenRestartWithRolling() throws Exception {
     Properties props = new Properties();
     String baseLogName = "biscuits";
@@ -367,7 +372,8 @@ public class CacheLogRollDUnitTest extends CacheTestCase {
     // Reenable this assertion once this issue (bug 42176) is fixed.
     assertTrue(f1c3.exists());
   }
-  
+
+  @Category(FlakyTest.class) // GEODE-676: possible disk pollution, file size sensitive
   public void testLogFileLayoutAndRolling() throws Exception {
     String baseLogName = "tacos";
       Properties props = new Properties();
@@ -401,6 +407,7 @@ public class CacheLogRollDUnitTest extends CacheTestCase {
       
   }
 
+  @Category(FlakyTest.class) // GEODE-675: possible disk pollution, file size sensitive
   public void testSecurityLogFileLayoutAndRolling() throws Exception {
     String baseLogName = "securitytacos";
       Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
index 639f93f..46e91b0 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
@@ -30,6 +30,8 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.InternalGemFireException;
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.Statistics;
@@ -66,6 +68,7 @@ import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Tests the ClientMembership API including ClientMembershipListener.
@@ -741,6 +744,7 @@ public class ClientMembershipDUnitTest extends ClientServerTestCase {
    * server joins when the client connects to the server. If the server
    * crashes or departs gracefully, the client will detect this as a crash.
    */
+  @Category(FlakyTest.class) // GEODE-1240: eats exceptions, random ports, time sensitive waits
   public void testClientMembershipEventsInClient() throws Exception {
     getSystem();
     IgnoredException.addIgnoredException("IOException");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckAsyncRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckAsyncRegionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckAsyncRegionDUnitTest.java
index 464b61c..4677985 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckAsyncRegionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckAsyncRegionDUnitTest.java
@@ -20,10 +20,6 @@ import com.gemstone.gemfire.cache.*;
 import java.io.*;
 import com.gemstone.gemfire.internal.OSProcess;
 
-/**
- *
- *
- */
 public class DiskDistributedNoAckAsyncRegionDUnitTest extends DiskDistributedNoAckRegionTestCase {
   
   /** Creates a new instance of DiskDistributedNoAckSyncOverflowRegionTest */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckRegionTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckRegionTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckRegionTestCase.java
index 426a8da..ff6bc67 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckRegionTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DiskDistributedNoAckRegionTestCase.java
@@ -18,11 +18,8 @@ package com.gemstone.gemfire.cache30;
 
 import com.gemstone.gemfire.cache.*;
 
-/**
- *
- *
- */
 public abstract class DiskDistributedNoAckRegionTestCase extends DistributedNoAckRegionDUnitTest {
+
   final protected DiskRegionTestImpl regionTestImpl;
   
   /** Creates a new instance of DiskDistributedNoAckRegionTest */
@@ -36,6 +33,6 @@ public abstract class DiskDistributedNoAckRegionTestCase extends DistributedNoAc
   }
   
   public void testBackupFillInValues() throws CacheException {
-        this.regionTestImpl.testBackupFillValues();
+    this.regionTestImpl.testBackupFillValues();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
index 27f5e36..6050e3d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckPersistentRegionCCEDUnitTest.java
@@ -1,20 +1,20 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 /**
  * 
  */
@@ -35,26 +35,20 @@ import com.gemstone.gemfire.internal.cache.DistributedRegion;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.SerializableCallable;
-import com.gemstone.gemfire.test.dunit.SerializableRunnable;
-import com.gemstone.gemfire.test.dunit.VM;
-
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.SerializableCallable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
+
 import java.io.IOException;
 import java.util.Map;
-
+
 import junit.framework.Assert;
 
-/**
- *
- */
 public class DistributedAckPersistentRegionCCEDUnitTest extends DistributedAckRegionCCEDUnitTest {
 
-  /**
-   * @param name
-   */
   public DistributedAckPersistentRegionCCEDUnitTest(String name) {
     super(name);
   }
@@ -70,18 +64,18 @@ public class DistributedAckPersistentRegionCCEDUnitTest extends DistributedAckRe
     return factory.create();
   }
 
-//  public void testClearWithConcurrentEventsAsync() throws Exception {
-//    int end = 100;
-//    for (int i=0; i<end; i++) {
-//      System.out.println("Starting run #" + i);
-//      super.testClearWithConcurrentEventsAsync();
-//      if (i<(end-1)) {
-//        tearDown();
-//        setUp();
-//      }
-//    }
-//  }
-  
+//  public void testClearWithConcurrentEventsAsync() throws Exception {
+//    int end = 100;
+//    for (int i=0; i<end; i++) {
+//      System.out.println("Starting run #" + i);
+//      super.testClearWithConcurrentEventsAsync();
+//      if (i<(end-1)) {
+//        tearDown();
+//        setUp();
+//      }
+//    }
+//  }
+  
   public void testClearOnNonReplicateWithConcurrentEvents() {}
   
   public void testConcurrentEventsOnNonReplicatedRegion() {}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java
index c9b9ac8..d642888 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionCCEDUnitTest.java
@@ -1,20 +1,20 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 /**
  * 
  */
@@ -24,9 +24,10 @@ import java.net.UnknownHostException;
 import java.util.Collections;
 import java.util.Properties;
 import java.util.Set;
-
+
 import junit.framework.Assert;
-
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.DataPolicy;
@@ -40,7 +41,7 @@ import com.gemstone.gemfire.cache.util.GatewayConflictResolver;
 import com.gemstone.gemfire.cache.util.TimestampedEntryEvent;
 import com.gemstone.gemfire.distributed.internal.DistributionAdvisor;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.NetMember;
 import com.gemstone.gemfire.internal.cache.AbstractRegionEntry;
@@ -60,25 +61,19 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.versions.VMVersionTag;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.cache.vmotion.VMotionObserver;
-import com.gemstone.gemfire.internal.cache.vmotion.VMotionObserverHolder;
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.SerializableCallable;
-import com.gemstone.gemfire.test.dunit.SerializableRunnable;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.internal.cache.vmotion.VMotionObserverHolder;
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.dunit.DistributedTestCase;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.SerializableCallable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
-/**
- *
- */
 public class DistributedAckRegionCCEDUnitTest extends DistributedAckRegionDUnitTest {
 
-  
-  /**
-   * @param name
-   */
   public DistributedAckRegionCCEDUnitTest(String name) {
     super(name);
   }
@@ -249,7 +244,7 @@ public class DistributedAckRegionCCEDUnitTest extends DistributedAckRegionDUnitT
         NetMember nm = CCRegion.getDistributionManager().getDistributionManagerId().getNetMember();
         InternalDistributedMember mbr = null;
         try {
-          mbr = new InternalDistributedMember(nm.getInetAddress().getCanonicalHostName(), nm.getPort()-1,
+          mbr = new InternalDistributedMember(nm.getInetAddress().getCanonicalHostName(), nm.getPort()-1,
               "fake_id", "fake_id_ustring", DistributionManager.NORMAL_DM_TYPE, null, null);
           tag.setMemberID(mbr);
         } catch (UnknownHostException e) {
@@ -287,11 +282,12 @@ public class DistributedAckRegionCCEDUnitTest extends DistributedAckRegionDUnitT
     versionTestConcurrentEvents();
   }
   
-  
+  @Category(FlakyTest.class) // GEODE-720: time sensitive, async actions, thread sleeps
   public void testClearWithConcurrentEvents() throws Exception {
     z_versionTestClearWithConcurrentEvents(true);
   }
 
+  @Category(FlakyTest.class) // GEODE-599 and GEODE-1046: async actions, thread sleeps -- // GEODE-1046: this may be hitting a product bug!
   public void testClearWithConcurrentEventsAsync() throws Exception {
     versionTestClearWithConcurrentEventsAsync();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java
index 26f0983..865761f 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedAckRegionDUnitTest.java
@@ -39,8 +39,6 @@ import com.gemstone.gemfire.test.dunit.VM;
  */
 public class DistributedAckRegionDUnitTest extends MultiVMRegionTestCase {
 
-
-  
   public DistributedAckRegionDUnitTest(String name) {
     super(name);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java
index f6836ae..9e9348b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionCCEDUnitTest.java
@@ -1,24 +1,26 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package com.gemstone.gemfire.cache30;
 
 import java.util.Map;
 import java.util.Properties;
-
+
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.CacheListener;
@@ -31,18 +33,18 @@ import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
-import com.gemstone.gemfire.test.dunit.SerializableRunnable;
-import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.test.dunit.Assert;
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.LogWriterUtils;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
-public class DistributedNoAckRegionCCEDUnitTest extends
-    DistributedNoAckRegionDUnitTest {
+public class DistributedNoAckRegionCCEDUnitTest extends DistributedNoAckRegionDUnitTest {
   
   static volatile boolean ListenerBlocking;
 
@@ -268,14 +270,14 @@ public class DistributedNoAckRegionCCEDUnitTest extends
   }
   
   
-  public void testTombstones() throws Exception {
-//    for (int i=0; i<1000; i++) {
+  public void testTombstones() throws Exception {
+//    for (int i=0; i<1000; i++) {
 //      System.out.println("starting run #"+i);
-      versionTestTombstones();
-//      if (i < 999) {
-//        tearDown();
-//        setUp();
-//      }
+      versionTestTombstones();
+//      if (i < 999) {
+//        tearDown();
+//        setUp();
+//      }
 //    }
   }
   
@@ -361,13 +363,11 @@ public class DistributedNoAckRegionCCEDUnitTest extends
    * This tests the concurrency versioning system to ensure that event conflation
    * happens correctly and that the statistic is being updated properly
    */
+  @Category(FlakyTest.class) // GEODE-976: time sensitive, thread sleeps, relies on stat values
   public void testConcurrentEventsOnEmptyRegion() {
     versionTestConcurrentEventsOnEmptyRegion();
   }
-  
-  
-  
-  
+
   /**
    * This tests the concurrency versioning system to ensure that event conflation
    * happens correctly and that the statistic is being updated properly

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionDUnitTest.java
index 538132c..64c4308 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/DistributedNoAckRegionDUnitTest.java
@@ -45,8 +45,7 @@ import com.gemstone.gemfire.test.dunit.WaitCriterion;
  *
  * @since 3.0
  */
-public class DistributedNoAckRegionDUnitTest
-  extends MultiVMRegionTestCase {
+public class DistributedNoAckRegionDUnitTest extends MultiVMRegionTestCase {
 
   public DistributedNoAckRegionDUnitTest(String name) {
     super(name);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
index 8d6160e..04662ca 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
@@ -47,6 +47,7 @@ import junit.framework.AssertionFailedError;
 
 import org.apache.logging.log4j.Logger;
 import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.DataSerializable;
 import com.gemstone.gemfire.DataSerializer;
@@ -127,7 +128,7 @@ import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
 import com.gemstone.gemfire.test.dunit.Invoke;
-
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Abstract superclass of {@link Region} tests that involve more than
@@ -1164,6 +1165,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
   /**
    * Tests that a {@link CacheListener} is invoked in a remote VM.
    */
+  @Category(FlakyTest.class) // GEODE-153 & GEODE-932: time sensitive, waitForInvocation (waitForCriterion), 3 second timeouts
   public void testRemoteCacheListener() throws InterruptedException {
     assertTrue(getRegionAttributes().getScope().isDistributed());
 
@@ -3840,8 +3842,8 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
    * Tests that an entry in a distributed region that expires with a distributed
    * destroy causes an event in other VM with isExpiration flag set.
    */
-    public void testEntryTtlDestroyEvent()
-    throws InterruptedException {
+  @Category(FlakyTest.class) // GEODE-583: time sensitive, expiration, waitForCriterion, short timeouts
+  public void testEntryTtlDestroyEvent() throws InterruptedException {
       
       if(getRegionAttributes().getPartitionAttributes() != null)
         return;
@@ -3994,11 +3996,11 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
     }
 
   /**
-     * Tests that an entry in a distributed region expires with a local
-     * destroy after a given time to live.
-     */
-    public void testEntryTtlLocalDestroy()
-    throws InterruptedException {
+   * Tests that an entry in a distributed region expires with a local
+   * destroy after a given time to live.
+   */
+  @Category(FlakyTest.class) // GEODE-671: time sensitive, expiration, retry loop, async actions, waitForCriterion
+  public void testEntryTtlLocalDestroy() throws InterruptedException {
       if(getRegionAttributes().getPartitionAttributes() != null)
         return;
       final boolean mirrored = getRegionAttributes().getDataPolicy().withReplication();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionDUnitTest.java
index 1599271..e46a223 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/PartitionedRegionDUnitTest.java
@@ -22,6 +22,8 @@ import java.util.Iterator;
 import java.util.Random;
 import java.util.Set;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
@@ -46,6 +48,7 @@ import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * This class tests the functionality of a cache {@link Region region}
@@ -309,6 +312,7 @@ public class PartitionedRegionDUnitTest extends MultiVMRegionTestCase {
   /**
    * test with multiple vms and a decent spread of keys
    */
+  @Category(FlakyTest.class) // GEODE-555: retry loops, use of Random
   public void testExtendedKeysValues() {
     final String regionName = getUniqueName();
     final int numEntries = 20000;
@@ -354,7 +358,7 @@ public class PartitionedRegionDUnitTest extends MultiVMRegionTestCase {
               assertTrue("should have been end of keys iteration", !keysIt.hasNext());
               assertTrue("should have been end of entries iteration", !entriesIt.hasNext());
             }
-            catch (Exception ex) {
+            catch (Exception ex) { // TODO: remove all of this and just disconnect DS in tear down
               try {
                 getRootRegion().getSubregion(regionName).destroyRegion();
               }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionReliabilityTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionReliabilityTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionReliabilityTestCase.java
index 1da697f..30bf72c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionReliabilityTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionReliabilityTestCase.java
@@ -25,6 +25,8 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.locks.Lock;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.AttributesMutator;
 import com.gemstone.gemfire.cache.CacheException;
@@ -71,6 +73,7 @@ import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
 import com.gemstone.gemfire.test.dunit.ThreadUtils;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Tests region reliability defined by MembershipAttributes.
@@ -953,6 +956,7 @@ public abstract class RegionReliabilityTestCase extends ReliabilityTestCase {
   /**
    * Tests affect of FULL_ACCESS on local entry expiration actions.
    */
+  @Category(FlakyTest.class) // GEODE-447: time sensitive, expiration, waitForMemberTimeout is unimplemented
   public void testFullAccessWithLocalEntryExpiration() throws Exception {
     final String name = this.getUniqueName();
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
index 64ca3ba..e2e270c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
@@ -32,6 +32,8 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.AttributesMutator;
@@ -63,10 +65,8 @@ import com.gemstone.gemfire.test.dunit.Invoke;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
-//import com.gemstone.gemfire.internal.util.DebuggerSupport;
-
-// @todo davidw Test {@link CacheStatistics}
 /**
  * An abstract class whose test methods test the functionality of a
  * region regardless of its scope.
@@ -76,6 +76,7 @@ import com.gemstone.gemfire.test.dunit.WaitCriterion;
  * This class also contains functionality that is used by subclasses.
  * See {@link #getRegionAttributes}.
  *
+ * TODO:davidw: Test {@link CacheStatistics}
  *
  * @since 3.0
  */
@@ -3506,8 +3507,8 @@ public abstract class RegionTestCase extends CacheTestCase {
    * Tests that an entry in a region that remains idle for a
    * given amount of time is destroyed.
    */
-  public void testEntryIdleDestroy()
-  throws CacheException, InterruptedException {
+  @Category(FlakyTest.class) // GEODE-706: time sensitive, expiration, waitForDestroy, EXPIRY_MS_PROPERTY, short timeout
+  public void testEntryIdleDestroy() throws Exception {
 
     final String name = this.getUniqueName();
     final int timeout = 20; // ms

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java
index a1d6aca..3af5447 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/TXDistributedDUnitTest.java
@@ -29,8 +29,6 @@
 
 package com.gemstone.gemfire.cache30;
 
-
-//import com.gemstone.gemfire.*;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.HashSet;
@@ -41,6 +39,7 @@ import java.util.concurrent.CountDownLatch;
 import junit.framework.AssertionFailedError;
 
 import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.AttributesFactory;
@@ -89,6 +88,7 @@ import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 public class TXDistributedDUnitTest extends CacheTestCase {
   public TXDistributedDUnitTest(String name) {
@@ -530,6 +530,7 @@ public class TXDistributedDUnitTest extends CacheTestCase {
     return p;
   }
 
+  @Category(FlakyTest.class) // GEODE-635: eats and logs exceptions, retry loops
   public void testHighAvailabilityFeatures() throws Exception {
     IgnoredException.addIgnoredException("DistributedSystemDisconnectedException");
 //    final CacheTransactionManager txMgr = this.getCache().getCacheTransactionManager();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
index 8dfe7a5..3b5f23f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
@@ -54,6 +54,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 import java.util.Set;
+
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
@@ -67,6 +70,7 @@ import com.gemstone.gemfire.test.dunit.ThreadUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Tests the ability of the {@link Locator} API to start and stop
@@ -1436,6 +1440,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
    * end up only have 1 master.
    * GEODE-870
    */
+  @Category(FlakyTest.class) // GEODE-1150: random ports, disk pollution, waitForCriterion, time sensitive, eats exceptions (fixed several)
   public void testMultipleLocatorsRestartingAtSameTime() throws Exception {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
@@ -1494,7 +1499,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
               return system.getDM().getViewMembers().size() == 6;
             } catch (Exception e) {
               e.printStackTrace();
-              fail("unexpected exception");
+              com.gemstone.gemfire.test.dunit.Assert.fail("unexpected exception", e);
             }
             return false; // NOTREACHED
           }
@@ -1518,7 +1523,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
               return system.getDM().getAllHostedLocators().size() == 0;
             } catch (Exception e) {
               e.printStackTrace();
-              fail("unexpected exception");
+              com.gemstone.gemfire.test.dunit.Assert.fail("unexpected exception", e);
             }
             return false; // NOTREACHED
           }
@@ -1546,7 +1551,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
               return system.getDM().getAllHostedLocators().size() == 2;
             } catch (Exception e) {
               e.printStackTrace();
-              fail("unexpected exception");
+              com.gemstone.gemfire.test.dunit.Assert.fail("unexpected exception", e);
             }
             return false; // NOTREACHED
           }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f12ece59/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
index 52ebe24..63c7c74 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
@@ -48,6 +48,7 @@ import com.gemstone.gemfire.internal.process.ProcessControllerFactory;
 import com.gemstone.gemfire.internal.process.ProcessStreamReader;
 import com.gemstone.gemfire.internal.process.ProcessType;
 import com.gemstone.gemfire.internal.process.ProcessUtils;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
@@ -146,6 +147,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
   }
   */
 
+  @Category(FlakyTest.class) // GEODE-473: random ports, BindException, forks JVM, uses ErrorCollector
   @Test
   public void testStartCreatesPidFile() throws Throwable {
     // build and start the locator
@@ -202,6 +204,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     }
   }
 
+  @Category(FlakyTest.class) // GEODE-530: BindException, random ports
   @Test
   public void testStartDeletesStaleControlFiles() throws Throwable {
     // create existing control files
@@ -272,7 +275,8 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
       this.errorCollector.addError(e);
     }
   }
-  
+
+  @Category(FlakyTest.class) // GEODE-1229: BindException
   @Test
   public void testStartOverwritesStalePidFile() throws Throwable {
     // create existing pid file
@@ -328,6 +332,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     }
   }
 
+  @Category(FlakyTest.class) // GEODE-764: BindException
   @Test
   public void testStartUsingForceOverwritesExistingPidFile() throws Throwable {
     // create existing pid file
@@ -691,7 +696,8 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
       this.errorCollector.addError(e);
     }
   }
-  
+
+  @Category(FlakyTest.class) // GEODE-569: BindException, random ports
   @Test
   public void testStatusUsingWorkingDirectory() throws Throwable {
     final List<String> jvmArguments = getJvmArguments();
@@ -890,7 +896,8 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
       this.errorCollector.addError(e);
     }
   }
-  
+
+  @Category(FlakyTest.class) // GEODE-847: random ports, BindException, forks JVM, uses ErrorCollector
   @Test
   public void testStopUsingWorkingDirectory() throws Throwable {
     final List<String> jvmArguments = getJvmArguments();