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/15 05:07:30 UTC

[01/32] incubator-geode git commit: GEODE-1178 Unexpected DistributedSystemDisconnectedException caused by RejectedExecutionException

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-17-2 2322827b0 -> 151472c70


GEODE-1178 Unexpected DistributedSystemDisconnectedException caused by RejectedExecutionException

This has been reported to JGroups.  While they're deciding what to do about
it I have coded a workaround in our StatRecorder class.  StatRecorder sits
in the JGroups stack just above the transport protocol that is throwing this
exception from its down() method.  StatRecorder will now catch the exception
and, after sleeping a short amount of time (10ms) it will retry as long as
the Manager is not shutting down.


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 39e94bc8beb22b62ed727640bbad3511affc9923
Parents: 86ab7cd
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Apr 12 10:43:26 2016 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Apr 12 10:45:40 2016 -0700

----------------------------------------------------------------------
 .../internal/membership/gms/Services.java       | 23 +++++--
 .../membership/gms/fd/GMSHealthMonitor.java     |  1 -
 .../gms/messenger/JGroupsMessenger.java         |  2 +-
 .../membership/gms/messenger/StatRecorder.java  | 30 +++++++--
 .../gms/membership/StatRecorderJUnitTest.java   | 67 ++++++++++++++------
 5 files changed, 93 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/39e94bc8/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
index 4484c00..4f5a1a4 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/Services.java
@@ -102,6 +102,19 @@ public class Services {
   }
   
 
+  /**
+   * for testing only - create a non-functional Services object with a Stopper
+   */
+  public Services() {
+    this.cancelCriterion = new Stopper();
+    this.stats = null;
+    this.config = null;
+    this.manager = null;
+    this.joinLeave = null;
+    this.healthMon = null;
+    this.messenger = null;
+    this.auth = null;
+  }
 
   public Services(
       DistributedMembershipListener listener, DistributionConfig config,
@@ -348,10 +361,10 @@ public class Services {
   public boolean isAutoReconnectEnabled() {
     return !getConfig().getDistributionConfig().getDisableAutoReconnect();
   }
-  
+
   public class Stopper extends CancelCriterion {
     volatile String reasonForStopping = null;
-    
+
     public void cancel(String reason) {
       this.reasonForStopping = reason;
     }
@@ -362,7 +375,7 @@ public class Services {
         return Services.this.shutdownCause.toString();
       return reasonForStopping;
     }
-    
+
     public boolean isCancelInProgress() {
       return cancelInProgress() != null;
     }
@@ -381,7 +394,7 @@ public class Services {
         }
       }
     }
-    
+
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/39e94bc8/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
index 510c5a8..5427d77 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
@@ -1190,7 +1190,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
    * @param initiator
    * @param sMembers
    * @param cv
-   * @param initiateRemoval
    */
   private void checkIfAvailable(final InternalDistributedMember initiator,
       List<SuspectRequest> sMembers, final NetView cv) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/39e94bc8/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index 2dfeeaa..a9abea3 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -293,7 +293,7 @@ public class JGroupsMessenger implements Messenger {
     // give the stats to the jchannel statistics recorder
     StatRecorder sr = (StatRecorder)myChannel.getProtocolStack().findProtocol(StatRecorder.class);
     if (sr != null) {
-      sr.setDMStats(services.getStatistics());
+      sr.setServices(services);
     }
     
     Transport transport = (Transport)myChannel.getProtocolStack().getTransport();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/39e94bc8/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/StatRecorder.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/StatRecorder.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/StatRecorder.java
index e29d71e..e013de6 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/StatRecorder.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/StatRecorder.java
@@ -32,6 +32,8 @@ import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.membership.gms.GMSUtil;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
 
+import java.util.concurrent.RejectedExecutionException;
+
 /**
  * JGroups doesn't capture quite the stats we want so this protocol is
  * inserted into the stack to gather the missing ones.
@@ -46,17 +48,19 @@ public class StatRecorder extends Protocol {
   private static final int INCOMING = 1;
   
   DMStats stats;
+  Services services;
   
   private final short nakackHeaderId = ClassConfigurator.getProtocolId(NAKACK2.class);
   private final short unicastHeaderId = ClassConfigurator.getProtocolId(UNICAST3.class);
   private final short frag2HeaderId = ClassConfigurator.getProtocolId(FRAG2.class);
   
   /**
-   * set the statistics object to modify when events are detected
-   * @param stats
+   * sets the services object of the GMS that is using this recorder
+   * @param services the Services collective of the GMS
    */
-  public void setDMStats(DMStats stats) {
-    this.stats = stats;
+  public void setServices(Services services) {
+    this.services = services;
+    this.stats = services.getStatistics();
   }
   
   @Override
@@ -81,7 +85,23 @@ public class StatRecorder extends Protocol {
       filter(msg, OUTGOING);
       break;
     }
-    return down_prot.down(evt);
+    do {
+      try {
+        return down_prot.down(evt);
+      } catch (RejectedExecutionException e) {
+        logger.debug("retrying JGroups message transmission due to rejected execution (GEODE-1178)");
+        try {
+          Thread.sleep(10);
+        } catch (InterruptedException ie) {
+          // down() does not throw InterruptedException so we can only set the interrupt flag and return
+          Thread.currentThread().interrupt();
+          return null;
+        }
+      }
+    } while (services != null
+      && !services.getManager().shutdownInProgress()
+      && !services.getCancelCriterion().isCancelInProgress());
+    return null;
   }
   
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/39e94bc8/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java
index 91d4a57..b7b80ac 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/StatRecorderJUnitTest.java
@@ -16,33 +16,34 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.membership;
 
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-
-import java.util.Properties;
-
-import org.jgroups.Event;
-import org.jgroups.Message;
-import org.jgroups.protocols.UNICAST3.Header;
-import org.jgroups.protocols.pbcast.NakAckHeader2;
-import org.jgroups.stack.Protocol;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.LonerDistributionManager.DummyDMStats;
 import com.gemstone.gemfire.distributed.internal.membership.gms.ServiceConfig;
 import com.gemstone.gemfire.distributed.internal.membership.gms.Services;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messenger.JGroupsMessenger;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messenger.StatRecorder;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.jgroups.Event;
+import org.jgroups.Message;
+import org.jgroups.protocols.UNICAST3.Header;
+import org.jgroups.protocols.pbcast.NakAckHeader2;
+import org.jgroups.stack.Protocol;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Properties;
+import java.util.concurrent.RejectedExecutionException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.*;
 
 /**
  * This class tests the GMS StatRecorder class, which records JGroups
@@ -53,14 +54,17 @@ public class StatRecorderJUnitTest {
   Protocol mockDownProtocol, mockUpProtocol;
   StatRecorder recorder;
   MyStats stats = new MyStats();
+  Services services;
   
   @Before
   public void initMocks() throws Exception {
     // create a StatRecorder that has mock up/down protocols and stats
     mockDownProtocol = mock(Protocol.class);
     mockUpProtocol = mock(Protocol.class);
+    services = mock(Services.class);
+    when(services.getStatistics()).thenReturn(stats);
     recorder = new StatRecorder();
-    recorder.setDMStats(stats);
+    recorder.setServices(services);
     recorder.setUpProtocol(mockUpProtocol);
     recorder.setDownProtocol(mockDownProtocol);
   }
@@ -94,6 +98,33 @@ public class StatRecorderJUnitTest {
         stats.ucastRetransmits == 1);
   }
 
+
+  @Test
+  public void recorderHandlesRejectedExecution() throws Exception {
+    Message msg = mock(Message.class);
+    when(msg.getHeader(any(Short.class))).thenReturn(Header.createDataHeader(1L, (short)1, true));
+    when(msg.size()).thenReturn(150L);
+
+
+    // GEODE-1178, the TP protocol may throw a RejectedExecutionException & StatRecorder should retry
+    when(mockDownProtocol.down(any(Event.class))).thenThrow(new RejectedExecutionException());
+
+    // after the first down() throws an exception we want StatRecorder to retry, so
+    // we set the Manager to say no shutdown is in progress the first time and then say
+    // one IS in progress so we can break out of the StatRecorder exception handling loop
+    when(services.getCancelCriterion()).thenReturn(new Services().getCancelCriterion());
+    Manager manager = mock(Manager.class);
+    when(services.getManager()).thenReturn(manager);
+    when(manager.shutdownInProgress()).thenReturn(Boolean.FALSE, Boolean.TRUE);
+
+    verify(mockDownProtocol, never()).down(isA(Event.class));
+
+    Event evt = new Event(Event.MSG, msg);
+    recorder.down(evt);
+
+    verify(mockDownProtocol, times(2)).down(isA(Event.class));
+  }
+
   /**
    * ensure that multicast events are recorded in DMStats
    */


[07/32] incubator-geode git commit: GEODE-1199: fix off-heap leak in netWrite

Posted by ji...@apache.org.
GEODE-1199: fix off-heap leak in netWrite


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 80533ba8503e8fa04aed3291b2d4efbc29e61535
Parents: fbee35c
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Apr 7 16:20:18 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Apr 13 10:16:20 2016 -0700

----------------------------------------------------------------------
 .../cache/SearchLoadAndWriteProcessor.java      | 28 ++++++--
 .../cache/SearchLoadAndWriteProcessorTest.java  | 68 ++++++++++++++++++++
 2 files changed, 90 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/80533ba8/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
index 8224fc2..d9729a7 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
@@ -74,6 +74,8 @@ import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.internal.offheap.annotations.Released;
+import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 
 
 /**
@@ -211,7 +213,7 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
 
     int action = paction;
     this.requestInProgress = true;
-    Scope scope = this.region.scope;
+    Scope scope = this.region.getScope();
     if (localWriter != null) {
       doLocalWrite(localWriter, event, action);
       this.requestInProgress = false;
@@ -220,14 +222,22 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
     if (scope == Scope.LOCAL && (region.getPartitionAttributes() == null)) {
       return false;
     }
+    @Released
     CacheEvent listenerEvent = getEventForListener(event);
+    try {
     if (action == BEFOREUPDATE && listenerEvent.getOperation().isCreate()) {
       action = BEFORECREATE;
     }
-    boolean cacheWrote = netWrite(getEventForListener(event), action, netWriteRecipients);
+    boolean cacheWrote = netWrite(listenerEvent, action, netWriteRecipients);
     this.requestInProgress = false;
     return cacheWrote;
-
+    } finally {
+      if (event != listenerEvent) {
+        if (listenerEvent instanceof EntryEventImpl) {
+          ((EntryEventImpl) listenerEvent).release();
+        }
+      }
+    }
   }
 
   public void memberJoined(InternalDistributedMember id) {
@@ -810,16 +820,20 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
 
   /**
    * Returns an event for listener notification.  The event's operation
-   * may be altered to conform to the ConcurrentMap implementation specification
+   * may be altered to conform to the ConcurrentMap implementation specification.
+   * If the returned value is not == to the event parameter then the caller
+   * is responsible for releasing it.
    * @param event the original event
    * @return the original event or a new event having a change in operation
    */
+  @Retained
   private CacheEvent getEventForListener(CacheEvent event) {
     Operation op = event.getOperation();
     if (!op.isEntry()) {
       return event;
     } else {
       EntryEventImpl r = (EntryEventImpl)event;
+      @Retained
       EntryEventImpl result = r;
       if (r.isSingleHop()) {
         // fix for bug #46130 - origin remote incorrect for one-hop operation in receiver
@@ -856,6 +870,7 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
         return false;
       }
     }
+    @Released
     CacheEvent event = getEventForListener(pevent);
     
     int action = paction;
@@ -898,8 +913,9 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
   /** Return true if cache writer was invoked */
   private boolean netWrite(CacheEvent event, int action, Set writeCandidateSet)
   throws CacheWriterException, TimeoutException {
-
-    // assert !writeCandidateSet.isEmpty();
+    if (writeCandidateSet == null || writeCandidateSet.isEmpty()) {
+      return false;
+    }
     ArrayList list = new ArrayList(writeCandidateSet);
     Collections.shuffle(list);
     InternalDistributedMember[] writeCandidates = (InternalDistributedMember[])list.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/80533ba8/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessorTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessorTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessorTest.java
new file mode 100644
index 0000000..57aca37
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessorTest.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class SearchLoadAndWriteProcessorTest {
+
+  /**
+   * This test verifies the fix for GEODE-1199.
+   * It verifies that when doNetWrite is called with an event
+   * that has a StoredObject value that it will have "release"
+   * called on it.
+   */
+  @Test
+  public void verifyThatOffHeapReleaseIsCalledAfterNetWrite() {
+    // setup
+    SearchLoadAndWriteProcessor processor = SearchLoadAndWriteProcessor.getProcessor();
+    LocalRegion lr = mock(LocalRegion.class);
+    when(lr.getScope()).thenReturn(Scope.DISTRIBUTED_ACK);
+    Object key = "key";
+    StoredObject value = mock(StoredObject.class);
+    when(value.hasRefCount()).thenReturn(true);
+    when(value.retain()).thenReturn(true);
+    Object cbArg = null;
+    KeyInfo keyInfo = new KeyInfo(key, value, cbArg);
+    when(lr.getKeyInfo(any(), any(), any())).thenReturn(keyInfo);
+    processor.region = lr;
+    EntryEventImpl event = EntryEventImpl.create(lr, Operation.REPLACE, key, value, cbArg, false, null);
+    
+    try {
+      // the test
+      processor.doNetWrite(event, null, null, 0);
+      
+      // verification
+      verify(value, times(2)).retain();
+      verify(value, times(1)).release();
+      
+    } finally {
+      processor.release();
+    }
+  }
+
+}


[02/32] incubator-geode git commit: GEODE-235: Add region name to EntryEventImpl toString

Posted by ji...@apache.org.
GEODE-235: Add region name to EntryEventImpl toString

A new EntryEventImplTest was added. It currently only
verifies that the toString output includes the region name.

This closes #125


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

Branch: refs/heads/feature/GEODE-17-2
Commit: c67a1c913d72062044767717078d8c5bcc5200a9
Parents: 39e94bc
Author: Scott Jewell <sj...@pivotal.io>
Authored: Tue Apr 5 11:03:06 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Apr 12 10:50:49 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/EntryEventImpl.java  |  4 +-
 .../internal/cache/EntryEventImplTest.java      | 71 ++++++++++++++++++++
 2 files changed, 74 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c67a1c91/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
index ba9ac11..3c87654 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
@@ -2306,6 +2306,8 @@ public class EntryEventImpl
 
     buf.append("op=");
     buf.append(getOperation());
+    buf.append(";region=");
+    buf.append(getRegion().getFullPath());
     buf.append(";key=");
     buf.append(this.getKey());
     buf.append(";oldValue=");
@@ -3121,4 +3123,4 @@ public class EntryEventImpl
   public final void setLoadedFromHDFS(boolean loadedFromHDFS) {
     this.loadedFromHDFS = loadedFromHDFS;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c67a1c91/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EntryEventImplTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EntryEventImplTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EntryEventImplTest.java
new file mode 100644
index 0000000..c3e057c
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/EntryEventImplTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import static org.mockito.Mockito.*;
+
+@Category(UnitTest.class)
+public class EntryEventImplTest {
+
+  String expectedRegionName = "ExpectedFullRegionPathName";
+  String key = "key1";
+  String value = "value1";
+  KeyInfo keyInfo = new KeyInfo(key, value, null);
+
+  @Test
+  public void verifyToStringOutputHasRegionName() {
+    // mock a region object
+    LocalRegion region = mock(LocalRegion.class);
+    doReturn(expectedRegionName).when(region).getFullPath();
+    doReturn(keyInfo).when(region).getKeyInfo(any(), any(), any());
+
+    // create entryevent for the region
+    EntryEventImpl e = createEntryEvent(region);
+    
+    // The name of the region should be in the toString text
+    String toStringValue = e.toString();
+    assertTrue("String " + expectedRegionName + " was not in toString text: " + toStringValue, toStringValue.indexOf(expectedRegionName) > 0);
+
+    // verify that toString called getFullPath method of region object
+    verify(region, Mockito.times(1)).getFullPath();
+  }
+
+  private EntryEventImpl createEntryEvent(LocalRegion l) {
+    // create a dummy event id
+    byte[] memId = { 1,2,3 };
+    EventID eventId = new EventID(memId, 11, 12, 13);
+
+    // create an event
+    EntryEventImpl event = EntryEventImpl.create(l, Operation.CREATE, key,
+        value, null,  false /* origin remote */, null,
+        false /* generateCallbacks */,
+        eventId);
+    // avoid calling invokeCallbacks
+    event.callbacksInvoked(true);
+
+    return event;
+  }
+}
\ No newline at end of file


[24/32] incubator-geode git commit: GEODE-1162: convert all CLI command DUnit tests to JUnit 4

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java
index 7cfb693..9844d90 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java
@@ -16,6 +16,26 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Invoke.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.Locator;
@@ -35,36 +55,21 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.remote.CommandProcessor;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.Invoke;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 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 org.apache.commons.io.FileUtils;
-
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Dunit class for testing GemFire config commands : export config
  *
  * @since 7.0
  */
+@Category(DistributedTest.class)
+@SuppressWarnings("serial")
 public class ConfigCommandsDUnitTest extends CliCommandTestBase {
-  private static final long serialVersionUID = 1L;
 
   File managerConfigFile = new File("Manager-cache.xml");
   File managerPropsFile = new File("Manager-gf.properties");
@@ -77,26 +82,23 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
   File subDir = new File("ConfigCommandsDUnitTestSubDir");
   File subManagerConfigFile = new File(subDir, managerConfigFile.getName());
 
-  public ConfigCommandsDUnitTest(String name) {
-    super(name);
-  }
-
   @Override
   protected void preTearDownCliCommandTestBase() throws Exception {
     deleteTestFiles();
-    Invoke.invokeInEveryVM(new SerializableRunnable() {
+    invokeInEveryVM(new SerializableRunnable() {
 
       @Override
       public void run() {
         try {
           deleteTestFiles();
         } catch (IOException e) {
-          Assert.fail("error", e);
+          fail("error", e);
         }
       }
     });
   }
 
+  @Test
   public void testDescribeConfig() throws ClassNotFoundException, IOException {
     createDefaultSetup(null);
     final String controllerName = "Member2";
@@ -123,10 +125,10 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
     List<String> jvmArgs = runtimeBean.getInputArguments();
 
-    LogWriterUtils.getLogWriter().info("#SB Actual JVM Args : ");
+    getLogWriter().info("#SB Actual JVM Args : ");
 
     for (String jvmArg : jvmArgs) {
-      LogWriterUtils.getLogWriter().info("#SB JVM " + jvmArg);
+      getLogWriter().info("#SB JVM " + jvmArg);
     }
 
     InternalDistributedSystem system = (InternalDistributedSystem) cache.getDistributedSystem();
@@ -140,7 +142,7 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     CommandResult cmdResult = executeCommand(command);
 
     String resultStr = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("#SB Hiding the defaults\n" + resultStr);
+    getLogWriter().info("#SB Hiding the defaults\n" + resultStr);
 
     assertEquals(true, cmdResult.getStatus().equals(Status.OK));
     assertEquals(true, resultStr.contains("G1"));
@@ -150,7 +152,7 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
 
     cmdResult = executeCommand(command + " --" + CliStrings.DESCRIBE_CONFIG__HIDE__DEFAULTS + "=false");
     resultStr = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("#SB No hiding of defaults\n" + resultStr);
+    getLogWriter().info("#SB No hiding of defaults\n" + resultStr);
 
     assertEquals(true, cmdResult.getStatus().equals(Status.OK));
     assertEquals(true, resultStr.contains("is-server"));
@@ -160,7 +162,7 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     cs.stop();
   }
 
-  @SuppressWarnings("serial")
+  @Test
   public void testExportConfig() throws IOException {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
@@ -252,12 +254,13 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
       FileReader reader = new FileReader(shellConfigFile);
       reader.read(fileContents);
     } catch (Exception ex) {
-      Assert.fail("Unable to read file contents for comparison", ex);
+      fail("Unable to read file contents for comparison", ex);
     }
 
     assertEquals(configToMatch, new String(fileContents));
   }
 
+  @Test
   public void testAlterRuntimeConfig() throws ClassNotFoundException, IOException {
     final String controller = "controller";
     createDefaultSetup(null);
@@ -279,8 +282,8 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, "10");
     CommandResult cmdResult = executeCommand(csb.getCommandString());
     String resultString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("Result\n");
-    LogWriterUtils.getLogWriter().info(resultString);
+    getLogWriter().info("Result\n");
+    getLogWriter().info(resultString);
     assertEquals(true, cmdResult.getStatus().equals(Status.OK));
     assertEquals(LogWriterImpl.INFO_LEVEL, config.getLogLevel());
     assertEquals(50, config.getLogFileSizeLimit());
@@ -295,6 +298,7 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     Result result = commandProcessor.createCommandStatement("alter runtime", Collections.EMPTY_MAP).process();
   }
 
+  @Test
   public void testAlterRuntimeConfigRandom() {
     final String member1 = "VM1";
     final String controller = "controller";
@@ -318,8 +322,8 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.ALTER_RUNTIME_CONFIG);
     CommandResult cmdResult = executeCommand(csb.getCommandString());
     String resultAsString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("#SB Result\n");
-    LogWriterUtils.getLogWriter().info(resultAsString);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultAsString);
     assertEquals(true, cmdResult.getStatus().equals(Status.ERROR));
     assertTrue(resultAsString.contains(CliStrings.ALTER_RUNTIME_CONFIG__RELEVANT__OPTION__MESSAGE));
 
@@ -327,12 +331,13 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, "2000000000");
     cmdResult = executeCommand(csb.getCommandString());
     resultAsString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("#SB Result\n");
-    LogWriterUtils.getLogWriter().info(resultAsString);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultAsString);
     assertEquals(true, cmdResult.getStatus().equals(Status.ERROR));
 
   }
 
+  @Test
   public void testAlterRuntimeConfigOnAllMembers() {
     final String member1 = "VM1";
     final String controller = "controller";
@@ -363,8 +368,8 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
     csb.addOption(CliStrings.ALTER_RUNTIME_CONFIG__LOG__DISK__SPACE__LIMIT, "10");
     CommandResult cmdResult = executeCommand(csb.getCommandString());
     String resultString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("#SB Result\n");
-    LogWriterUtils.getLogWriter().info(resultString);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultString);
     assertEquals(true, cmdResult.getStatus().equals(Status.OK));
     assertEquals(LogWriterImpl.INFO_LEVEL, config.getLogLevel());
     assertEquals(50, config.getLogFileSizeLimit());
@@ -428,7 +433,7 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
               return "Waiting for shared configuration to be started";
             }
           };
-          Wait.waitForCriterion(wc, 5000, 500, true);
+          waitForCriterion(wc, 5000, 500, true);
         } catch (IOException ioex) {
           fail("Unable to create a locator with a shared configuration");
         }
@@ -479,7 +484,7 @@ public class ConfigCommandsDUnitTest extends CliCommandTestBase {
           gemfireProperties = sharedConfig.getConfiguration(groupName).getGemfireProperties();
           assertEquals("fine", gemfireProperties.get(DistributionConfig.LOG_LEVEL_NAME));
         } catch (Exception e) {
-          Assert.fail("Error occurred in cluster configuration service", e);
+          fail("Error occurred in cluster configuration service", e);
         }
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
index 10b37c1..b6692c7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
@@ -16,6 +16,30 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.jayway.awaitility.Awaitility.*;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.text.MessageFormat;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
@@ -41,29 +65,17 @@ import com.gemstone.gemfire.management.internal.ManagementConstants;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.jayway.awaitility.Awaitility;
-
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.text.MessageFormat;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeUnit;
+import com.gemstone.gemfire.test.junit.Retry;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import com.gemstone.gemfire.test.junit.rules.RetryRule;
 
+@Category({ DistributedTest.class, FlakyTest.class }) // GEODE-973
 public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBase {
+
   private static final long serialVersionUID = 1L;
 
   final String alterRegionName = "testAlterRegionRegion";
@@ -81,13 +93,13 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
 
   final List<String> filesToBeDeleted = new CopyOnWriteArrayList<String>();
 
-  public CreateAlterDestroyRegionCommandsDUnitTest(String name) {
-    super(name);
-  }
+  @Rule
+  public RetryRule retryRule = new RetryRule();
 
   /**
    * Asserts that the "compressor" option for the "create region" command succeeds for a recognized compressor.
    */
+  @Test
   public void testCreateRegionWithGoodCompressor() {
     createDefaultSetup(null);
     VM vm = Host.getHost(0).getVM(1);
@@ -123,6 +135,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
   /**
    * Asserts that the "compressor" option for the "create region" command fails for an unrecognized compressorc.
    */
+  @Test
   public void testCreateRegionWithBadCompressor() {
     createDefaultSetup(null);
 
@@ -152,6 +165,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
    * Asserts that a missing "compressor" option for the "create region" command results in a region with no
    * compression.
    */
+  @Test
   public void testCreateRegionWithNoCompressor() {
     createDefaultSetup(null);
 
@@ -183,6 +197,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     assertEquals(Result.Status.OK, cmdResult.getStatus());
   }
 
+  @Test
   public void testDestroyDistributedRegion() {
     createDefaultSetup(null);
 
@@ -205,39 +220,40 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
 
     // Test failure when region not found
     String command = "destroy region --name=DOESNOTEXIST";
-    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    getLogWriter().info("testDestroyRegion command=" + command);
     CommandResult cmdResult = executeCommand(command);
     String strr = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
     assertTrue(stringContainsLine(strr, "Could not find.*\"DOESNOTEXIST\".*"));
     assertEquals(Result.Status.ERROR, cmdResult.getStatus());
 
     // Test unable to destroy with co-location
     command = "destroy region --name=/Customer";
-    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    getLogWriter().info("testDestroyRegion command=" + command);
     cmdResult = executeCommand(command);
     strr = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
     assertEquals(Result.Status.ERROR, cmdResult.getStatus());
 
     // Test success
     command = "destroy region --name=/Order";
-    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    getLogWriter().info("testDestroyRegion command=" + command);
     cmdResult = executeCommand(command);
     strr = commandResultToString(cmdResult);
     assertTrue(stringContainsLine(strr, ".*Order.*destroyed successfully.*"));
-    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
 
     command = "destroy region --name=/Customer";
-    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    getLogWriter().info("testDestroyRegion command=" + command);
     cmdResult = executeCommand(command);
     strr = commandResultToString(cmdResult);
     assertTrue(stringContainsLine(strr, ".*Customer.*destroyed successfully.*"));
-    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
   }
 
+  @Test
   public void testDestroyLocalRegions() {
     createDefaultSetup(null);
 
@@ -255,19 +271,19 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
 
     // Test failure when region not found
     String command = "destroy region --name=DOESNOTEXIST";
-    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    getLogWriter().info("testDestroyRegion command=" + command);
     CommandResult cmdResult = executeCommand(command);
     String strr = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
     assertTrue(stringContainsLine(strr, "Could not find.*\"DOESNOTEXIST\".*"));
     assertEquals(Result.Status.ERROR, cmdResult.getStatus());
 
     command = "destroy region --name=/Customer";
-    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    getLogWriter().info("testDestroyRegion command=" + command);
     cmdResult = executeCommand(command);
     strr = commandResultToString(cmdResult);
     assertTrue(stringContainsLine(strr, ".*Customer.*destroyed successfully.*"));
-    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
 
     for (int i = 1; i <= 3; i++) {
@@ -278,6 +294,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     }
   }
 
+  @Test
   public void testDestroyLocalAndDistributedRegions() {
     createDefaultSetup(null);
 
@@ -300,19 +317,19 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
 
     // Test failure when region not found
     String command = "destroy region --name=DOESNOTEXIST";
-    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    getLogWriter().info("testDestroyRegion command=" + command);
     CommandResult cmdResult = executeCommand(command);
     String strr = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
     assertTrue(stringContainsLine(strr, "Could not find.*\"DOESNOTEXIST\".*"));
     assertEquals(Result.Status.ERROR, cmdResult.getStatus());
 
     command = "destroy region --name=/Customer";
-    LogWriterUtils.getLogWriter().info("testDestroyRegion command=" + command);
+    getLogWriter().info("testDestroyRegion command=" + command);
     cmdResult = executeCommand(command);
     strr = commandResultToString(cmdResult);
     assertTrue(stringContainsLine(strr, ".*Customer.*destroyed successfully.*"));
-    LogWriterUtils.getLogWriter().info("testDestroyRegion strr=" + strr);
+    getLogWriter().info("testDestroyRegion strr=" + strr);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
 
     for (int i = 1; i <= 3; i++) {
@@ -325,7 +342,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
 
   private void waitForRegionMBeanCreation(final String regionPath, final int mbeanCount) {
     Host.getHost(0).getVM(0).invoke(() -> {
-      Awaitility.waitAtMost(5, TimeUnit.SECONDS).until(newRegionMBeanIsCreated(regionPath, mbeanCount));
+      waitAtMost(5, TimeUnit.SECONDS).until(newRegionMBeanIsCreated(regionPath, mbeanCount));
     });
   }
 
@@ -337,30 +354,32 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
         ObjectName queryExpON = new ObjectName(queryExp);
         return mbeanServer.queryNames(null, queryExpON).size() == mbeanCount;
       } catch (MalformedObjectNameException mone) {
-        LogWriterUtils.getLogWriter().error(mone);
+        getLogWriter().error(mone);
         fail(mone.getMessage());
         return false;
       }
     };
   }
 
+  @Test
+  @Retry(2) // GEODE-973: getRandomAvailablePort
   public void testCreateRegion46391() throws IOException {
-    createDefaultSetup(null);
+    createDefaultSetup(null); // GEODE-973: getRandomAvailablePort
     String command = CliStrings.CREATE_REGION + " --" + CliStrings.CREATE_REGION__REGION + "=" + this.region46391 + " --" + CliStrings.CREATE_REGION__REGIONSHORTCUT + "=REPLICATE";
 
-    LogWriterUtils.getLogWriter().info("testCreateRegion46391 create region command=" + command);
+    getLogWriter().info("testCreateRegion46391 create region command=" + command);
 
     CommandResult cmdResult = executeCommand(command);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
 
     command = CliStrings.PUT + " --" + CliStrings.PUT__KEY + "=k1" + " --" + CliStrings.PUT__VALUE + "=k1" + " --" + CliStrings.PUT__REGIONNAME + "=" + this.region46391;
 
-    LogWriterUtils.getLogWriter().info("testCreateRegion46391 put command=" + command);
+    getLogWriter().info("testCreateRegion46391 put command=" + command);
 
     CommandResult cmdResult2 = executeCommand(command);
     assertEquals(Result.Status.OK, cmdResult2.getStatus());
 
-    LogWriterUtils.getLogWriter().info("testCreateRegion46391  cmdResult2=" + commandResultToString(cmdResult2));
+    getLogWriter().info("testCreateRegion46391  cmdResult2=" + commandResultToString(cmdResult2));
     String str1 = "Result      : true";
     String str2 = "Key         : k1";
     String str3 = "Key Class   : java.lang.String";
@@ -376,7 +395,9 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     assertTrue(commandResultToString(cmdResult2).contains(str5));
   }
 
-  public void bug51924_testAlterRegion() throws IOException {
+  @Ignore("bug51924")
+  @Test
+  public void testAlterRegion() throws IOException {
     createDefaultSetup(null);
 
     CommandResult cmdResult = executeCommand(CliStrings.LIST_REGION);
@@ -748,6 +769,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
   /**
    * Asserts that creating, altering and destroying regions correctly updates the shared configuration.
    */
+  @Test
   public void testCreateAlterDestroyUpdatesSharedConfig() {
     disconnectAllFromDS();
 
@@ -768,7 +790,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
         final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
             locatorProps);
 
-        Awaitility.waitAtMost(5, TimeUnit.SECONDS).until(() -> locator.isSharedConfigurationRunning());
+        waitAtMost(5, TimeUnit.SECONDS).until(() -> locator.isSharedConfigurationRunning());
       } catch (IOException ioex) {
         fail("Unable to create a locator with a shared configuration");
       }
@@ -809,7 +831,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
       try {
         assertTrue(sharedConfig.getConfiguration(groupName).getCacheXmlContent().contains(regionName));
       } catch (Exception e) {
-        Assert.fail("Error in cluster configuration service", e);
+        fail("Error in cluster configuration service", e);
       }
     });
 
@@ -880,6 +902,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
 
   }
 
+  @Test
   public void testDestroyRegionWithSharedConfig() {
 
     disconnectAllFromDS();
@@ -901,7 +924,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
         final InternalLocator locator = (InternalLocator) Locator.startLocatorAndDS(locatorPort, locatorLogFile, null,
             locatorProps);
 
-        Awaitility.waitAtMost(5, TimeUnit.SECONDS).until(() -> locator.isSharedConfigurationRunning());
+        waitAtMost(5, TimeUnit.SECONDS).until(() -> locator.isSharedConfigurationRunning());
       } catch (IOException ioex) {
         fail("Unable to create a locator with a shared configuration");
       }
@@ -950,7 +973,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
     commandStringBuilder = new CommandStringBuilder(CliStrings.DESTROY_REGION);
     commandStringBuilder.addOption(CliStrings.DESTROY_REGION__REGION, regionName);
     cmdResult = executeCommand(commandStringBuilder.toString());
-    LogWriterUtils.getLogWriter().info("#SB" + commandResultToString(cmdResult));
+    getLogWriter().info("#SB" + commandResultToString(cmdResult));
     assertEquals(Result.Status.OK, cmdResult.getStatus());
 
     // Make sure the region was removed from the shared config
@@ -997,7 +1020,7 @@ public class CreateAlterDestroyRegionCommandsDUnitTest extends CliCommandTestBas
           executeCommand("undeploy --jar=" + fileToDelete.getName());
         }
       } catch (IOException e) {
-        LogWriterUtils.getLogWriter().error("Unable to delete file", e);
+        getLogWriter().error("Unable to delete file", e);
       }
     }
     this.filesToBeDeleted.clear();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
index b70b9fa..fee3978 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
@@ -16,6 +16,17 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.Properties;
+import java.util.regex.Pattern;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -35,31 +46,22 @@ 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 java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.util.Properties;
-import java.util.regex.Pattern;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Unit tests for the DeployCommands class
  *
  * @since 7.0
  */
+@Category(DistributedTest.class)
+@SuppressWarnings("serial")
 public class DeployCommandsDUnitTest extends CliCommandTestBase {
-  private static final long serialVersionUID = 1L;
 
   File newDeployableJarFile = new File("DeployCommandsDUnit1.jar");
 
   transient private ClassBuilder classBuilder = new ClassBuilder();
   transient private CommandProcessor commandProcessor;
 
-  public DeployCommandsDUnitTest(String name) {
-    super(name);
-  }
-
-  @SuppressWarnings("serial")
   @Override
   public final void postSetUp() throws Exception {
     this.commandProcessor = new CommandProcessor();
@@ -90,7 +92,7 @@ public class DeployCommandsDUnitTest extends CliCommandTestBase {
     deleteSavedJarFiles();
   }
 
-  @SuppressWarnings("serial")
+  @Test
   public void testDeploy() throws IOException {
     final Properties props = new Properties();
     final Host host = Host.getHost(0);
@@ -176,7 +178,7 @@ public class DeployCommandsDUnitTest extends CliCommandTestBase {
     assertEquals(2, countMatchesInString(resultString, "DeployCommandsDUnit6.jar"));
   }
 
-  @SuppressWarnings("serial")
+  @Test
   public void testUndeploy() throws IOException {
     final Properties props = new Properties();
     final Host host = Host.getHost(0);
@@ -253,7 +255,7 @@ public class DeployCommandsDUnitTest extends CliCommandTestBase {
     assertEquals(4, countMatchesInString(resultString, "DeployCommandsDUnit5.jar"));
   }
 
-  @SuppressWarnings("serial")
+  @Test
   public void testListDeployed() throws IOException {
     final Properties props = new Properties();
     final Host host = Host.getHost(0);
@@ -322,6 +324,7 @@ public class DeployCommandsDUnitTest extends CliCommandTestBase {
   /**
    * Does an end-to-end test using the complete CLI framework while ensuring that the shared configuration is updated.
    */
+  @Test
   public void testEndToEnd() throws IOException {
     final String groupName = "testDeployEndToEndGroup";
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
index 566977c..e8ee713 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
@@ -16,6 +16,23 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.StringTokenizer;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.DataPolicy;
@@ -46,25 +63,12 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 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 org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.StringTokenizer;
-import java.util.concurrent.CopyOnWriteArrayList;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * The DiskStoreCommandsDUnitTest class is a distributed test suite of test cases for testing the disk store commands
@@ -75,16 +79,12 @@ import java.util.concurrent.CopyOnWriteArrayList;
  * @see org.junit.Test
  * @since 7.0
  */
+@Category(DistributedTest.class)
+@SuppressWarnings("serial")
 public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
-  private static final long serialVersionUID = 1L;
 
   final List<String> filesToBeDeleted = new CopyOnWriteArrayList<String>();
 
-  public DiskStoreCommandsDUnitTest(final String testName) {
-    super(testName);
-  }
-
-  @SuppressWarnings("serial")
   @Test
   public void testMissingDiskStore() {
     final String regionName = "testShowMissingDiskStoreRegion";
@@ -198,7 +198,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
             return "Waiting for another persistent member to come online";
           }
         };
-        Wait.waitForCriterion(waitCriterion, 70000, 100, true);
+        waitForCriterion(waitCriterion, 70000, 100, true);
       }
     });
 
@@ -240,6 +240,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testDescribeOfflineDiskStore() {
     createDefaultSetup(null);
 
@@ -292,6 +293,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
     assertFalse(stringContainsLine(stringResult, ".*/" + region2 + ": .*"));
   }
 
+  @Test
   public void testOfflineDiskStorePdxCommands() {
     final Properties props = new Properties();
     props.setProperty("mcast-port", "0");
@@ -336,7 +338,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
         stringContainsLine(stringResult, ".*com\\.gemstone\\.gemfire\\.cache\\.query\\.data\\.PortfolioPdx\\$Day.*"));
   }
 
-
+  @Test
   public void testValidateDiskStore() {
     createDefaultSetup(null);
 
@@ -369,20 +371,21 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
       }
     });
     String command = "validate offline-disk-store --name=" + diskStoreName1 + " --disk-dirs=" + diskStoreDir.getAbsolutePath();
-    LogWriterUtils.getLogWriter().info("testValidateDiskStore command: " + command);
+    getLogWriter().info("testValidateDiskStore command: " + command);
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testValidateDiskStore cmdResult is stringResult " + stringResult);
+      getLogWriter().info("testValidateDiskStore cmdResult is stringResult " + stringResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       assertTrue(stringResult.contains("Total number of region entries in this disk store is"));
 
     } else {
-      LogWriterUtils.getLogWriter().info("testValidateDiskStore cmdResult is null");
+      getLogWriter().info("testValidateDiskStore cmdResult is null");
       fail("Did not get CommandResult in testValidateDiskStore");
     }
   }
 
+  @Test
   public void testExportOfflineDiskStore() throws Exception {
     createDefaultSetup(null);
 
@@ -423,7 +426,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
       }
     });
     String command = "export offline-disk-store --name=" + diskStoreName1 + " --disk-dirs=" + diskStoreDir.getAbsolutePath() + " --dir=" + exportDir;
-    LogWriterUtils.getLogWriter().info("testExportDiskStore command" + command);
+    getLogWriter().info("testExportDiskStore command" + command);
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -432,7 +435,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
       SnapshotTestUtil.checkSnapshotEntries(exportDir, entries, diskStoreName1, region2);
 
     } else {
-      LogWriterUtils.getLogWriter().info("testExportOfflineDiskStore cmdResult is null");
+      getLogWriter().info("testExportOfflineDiskStore cmdResult is null");
       fail("Did not get CommandResult in testExportOfflineDiskStore");
     }
   }
@@ -440,6 +443,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
   /**
    * Asserts that creating and destroying disk stores correctly updates the shared configuration.
    */
+  @Test
   public void testCreateDestroyUpdatesSharedConfig() {
     disconnectAllFromDS();
 
@@ -473,7 +477,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
               return "Waiting for shared configuration to be started";
             }
           };
-          Wait.waitForCriterion(wc, 5000, 500, true);
+          waitForCriterion(wc, 5000, 500, true);
         } catch (IOException ioex) {
           fail("Unable to create a locator with a shared configuration");
         }
@@ -522,7 +526,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
           xmlFromConfig = sharedConfig.getConfiguration(groupName).getCacheXmlContent();
           assertTrue(xmlFromConfig.contains(diskStoreName));
         } catch (Exception e) {
-          Assert.fail("Error occurred in cluster configuration service", e);
+          fail("Error occurred in cluster configuration service", e);
         }
       }
     });
@@ -573,7 +577,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
           xmlFromConfig = sharedConfig.getConfiguration(groupName).getCacheXmlContent();
           assertFalse(xmlFromConfig.contains(diskStoreName));
         } catch (Exception e) {
-          Assert.fail("Error occurred in cluster configuration service", e);
+          fail("Error occurred in cluster configuration service", e);
         }
       }
     });
@@ -603,13 +607,14 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
   }
 
 
-  /****
+  /**
    * 1) Create a disk-store in a member, get the disk-dirs. 2) Close the member. 3) Execute the command. 4) Restart the
    * member. 5) Check if the disk-store is altered.
    *
    * @throws IOException
    * @throws ClassNotFoundException
    */
+  @Test
   public void testAlterDiskStore() throws ClassNotFoundException, IOException {
     final String regionName = "region1";
     final String diskStoreName = "disk-store1";
@@ -664,7 +669,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
 
     CommandResult cmdResult = executeCommand(commandString);
     String resultString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("#SB command output : \n" + resultString);
+    getLogWriter().info("#SB command output : \n" + resultString);
     assertEquals(true, Result.Status.OK.equals(cmdResult.getStatus()));
     assertEquals(true, resultString.contains("concurrencyLevel=5"));
     assertEquals(true, resultString.contains("lruAction=local-destroy"));
@@ -695,7 +700,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
 
     cmdResult = executeCommand(commandString);
     resultString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("command output : \n" + resultString);
+    getLogWriter().info("command output : \n" + resultString);
     assertEquals(true, Result.Status.OK.equals(cmdResult.getStatus()));
 
     Object postDestroyValue = vm1.invoke(new SerializableCallable() {
@@ -722,13 +727,13 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
 
     cmdResult = executeCommand(commandString);
     resultString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("Alter DiskStore with wrong remove option  : \n" + resultString);
+    getLogWriter().info("Alter DiskStore with wrong remove option  : \n" + resultString);
     assertEquals(true, Result.Status.ERROR.equals(cmdResult.getStatus()));
 
     filesToBeDeleted.add(diskDirName);
   }
 
-
+  @Test
   public void testBackupDiskStoreBackup() throws IOException {
     final String regionName = "region1";
     final String fullBackUpName = "fullBackUp";
@@ -789,7 +794,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
 
     CommandResult cmdResult = executeCommand(commandString);
     String resultAsString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("Result from full backup : \n" + resultAsString);
+    getLogWriter().info("Result from full backup : \n" + resultAsString);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
     assertEquals(true, resultAsString.contains("Manager"));
     assertEquals(true, resultAsString.contains(vm1Name));
@@ -819,13 +824,14 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
 
     cmdResult = executeCommand(csb.toString());
     resultAsString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("Result from incremental backup : \n" + resultAsString);
+    getLogWriter().info("Result from incremental backup : \n" + resultAsString);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
 
     assertEquals(true, resultAsString.contains("Manager"));
     assertEquals(true, resultAsString.contains(vm1Name));
   }
 
+  @Test
   public void testCreateDiskStore() {
     final String diskStore1Name = "testCreateDiskStore1";
     final String diskStore2Name = "testCreateDiskStore2";
@@ -942,6 +948,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
     assertTrue(stringContainsLine(stringResult, vm2Name + ".*" + diskStore2Name + " .*"));
   }
 
+  @Test
   public void testDestroyDiskStore() {
     final String diskStore1Name = "testDestroyDiskStore1";
     final String diskStore2Name = "testDestroyDiskStore2";
@@ -1151,7 +1158,7 @@ public class DiskStoreCommandsDUnitTest extends CliCommandTestBase {
       try {
         deleteFiles();
       } catch (IOException e) {
-        LogWriterUtils.getLogWriter().error("Unable to delete file", e);
+        getLogWriter().error("Unable to delete file", e);
       }
     }
     this.filesToBeDeleted.clear();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java
index 4b9f4c6..788e9eb 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java
@@ -24,6 +24,15 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
+import org.jmock.Expectations;
+import org.jmock.Mockery;
+import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.execute.Execution;
 import com.gemstone.gemfire.cache.execute.FunctionInvocationTargetException;
@@ -31,7 +40,6 @@ import com.gemstone.gemfire.cache.execute.ResultCollector;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.cache.execute.AbstractExecution;
 import com.gemstone.gemfire.internal.util.CollectionUtils;
-import com.gemstone.gemfire.management.internal.cli.CliUtil;
 import com.gemstone.gemfire.management.internal.cli.domain.DiskStoreDetails;
 import com.gemstone.gemfire.management.internal.cli.functions.DescribeDiskStoreFunction;
 import com.gemstone.gemfire.management.internal.cli.functions.ListDiskStoresFunction;
@@ -40,15 +48,6 @@ import com.gemstone.gemfire.management.internal.cli.util.DiskStoreNotFoundExcept
 import com.gemstone.gemfire.management.internal.cli.util.MemberNotFoundException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.jmock.Expectations;
-import org.jmock.Mockery;
-import org.jmock.lib.legacy.ClassImposteriser;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 /**
  * The DiskStoreCommandsJUnitTest class is a test suite of test cases testing the contract and functionality of the
  * DiskStoreCommands class implementing commands in the GemFire shell (gfsh) that access and modify disk stores in

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
index 8451916..b4d4df1 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
@@ -16,6 +16,16 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.util.List;
+import java.util.Properties;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
@@ -32,30 +42,23 @@ import com.gemstone.gemfire.management.internal.cli.json.GfJsonException;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 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 java.util.List;
-import java.util.Properties;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Dunit class for testing gemfire function commands : execute function, destroy function, list function
- *
  */
+@Category(DistributedTest.class)
 public class FunctionCommandsDUnitTest extends CliCommandTestBase {
+
   private static final long serialVersionUID = 1L;
   private static final String REGION_NAME = "FunctionCommandsReplicatedRegion";
   private static final String REGION_ONE = "RegionOne";
   private static final String REGION_TWO = "RegionTwo";
 
-  public FunctionCommandsDUnitTest(String name) {
-    super(name);
-  }
-
   void setupWith2Regions() {
     final VM vm1 = Host.getHost(0).getVM(1);
     final VM vm2 = Host.getHost(0).getVM(2);
@@ -101,7 +104,8 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
     });
   }
 
-  public void testExecuteFunctionWithNoRegionOnManager() {
+  @Test
+  public void testExecuteFunctionWithNoRegionOnManager() throws Exception {
     setupWith2Regions();
     Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
     FunctionService.registerFunction(function);
@@ -111,18 +115,13 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
         FunctionService.registerFunction(function);
       }
     });
-    try {
-      Thread.sleep(2500);
-    } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
+    Thread.sleep(2500);
     String command = "execute function --id=" + function.getId() + " --region=" + "/" + "RegionOne";
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionWithNoRegionOnManager command : " + command);
+    getLogWriter().info("testExecuteFunctionWithNoRegionOnManager command : " + command);
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testExecuteFunctionWithNoRegionOnManager stringResult : " + strCmdResult);
+      getLogWriter().info("testExecuteFunctionWithNoRegionOnManager stringResult : " + strCmdResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       assertTrue(strCmdResult.contains("Execution summary"));
     } else {
@@ -132,10 +131,11 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
   }
 
   public static String getMemberId() {
-    Cache cache = new FunctionCommandsDUnitTest("test").getCache();
+    Cache cache = new FunctionCommandsDUnitTest().getCache(); // TODO: get rid of this kind of stuff
     return cache.getDistributedSystem().getDistributedMember().getId();
   }
 
+  @Test
   public void testExecuteFunctionOnRegion() {
     createDefaultSetup(null);
 
@@ -150,13 +150,13 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
     });
 
     String command = "execute function --id=" + function.getId() + " --region=" + REGION_NAME;
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionOnRegion command=" + command);
+    getLogWriter().info("testExecuteFunctionOnRegion command=" + command);
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       assertEquals(Result.Status.OK, cmdResult.getStatus());
-      LogWriterUtils.getLogWriter().info("testExecuteFunctionOnRegion cmdResult=" + cmdResult);
+      getLogWriter().info("testExecuteFunctionOnRegion cmdResult=" + cmdResult);
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testExecuteFunctionOnRegion stringResult=" + stringResult);
+      getLogWriter().info("testExecuteFunctionOnRegion stringResult=" + stringResult);
       assert (stringResult.contains("Execution summary"));
     } else {
       fail("testExecuteFunctionOnRegion did not return CommandResult");
@@ -193,7 +193,7 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
           if (bean == null) {
             return false;
           } else {
-            LogWriterUtils.getLogWriter().info("Probing for checkRegionMBeans testExecuteFunctionOnRegionBug51480 finished");
+            getLogWriter().info("Probing for checkRegionMBeans testExecuteFunctionOnRegionBug51480 finished");
             return true;
           }
         }
@@ -203,13 +203,14 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
           return "Probing for testExecuteFunctionOnRegionBug51480";
         }
       };
-      Wait.waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+      waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
       DistributedRegionMXBean bean = ManagementService.getManagementService(getCache()).getDistributedRegionMXBean(
           Region.SEPARATOR + REGION_ONE);
       assertNotNull(bean);
     }
   };
 
+  @Test
   public void testExecuteFunctionOnRegionBug51480() {
     setupForBug51480();
 
@@ -226,13 +227,13 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
 
     String command = "execute function --id=" + function.getId() + " --region=" + REGION_ONE;
 
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionOnRegionBug51480 command=" + command);
+    getLogWriter().info("testExecuteFunctionOnRegionBug51480 command=" + command);
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
-      LogWriterUtils.getLogWriter().info("testExecuteFunctionOnRegionBug51480 cmdResult=" + cmdResult);
+      getLogWriter().info("testExecuteFunctionOnRegionBug51480 cmdResult=" + cmdResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testExecuteFunctionOnRegionBug51480 stringResult=" + stringResult);
+      getLogWriter().info("testExecuteFunctionOnRegionBug51480 stringResult=" + stringResult);
       assert (stringResult.contains("Execution summary"));
     } else {
       fail("testExecuteFunctionOnRegionBug51480 did not return CommandResult");
@@ -240,6 +241,7 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testExecuteFunctionOnMember() {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
@@ -261,15 +263,16 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
     });
 
     String command = "execute function --id=" + function.getId() + " --member=" + vm1MemberId;
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionOnMember command=" + command);
+    getLogWriter().info("testExecuteFunctionOnMember command=" + command);
     CommandResult cmdResult = executeCommand(command);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
 
     String stringResult = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionOnMember stringResult:" + stringResult);
+    getLogWriter().info("testExecuteFunctionOnMember stringResult:" + stringResult);
     assertTrue(stringResult.contains("Execution summary"));
   }
 
+  @Test
   public void testExecuteFunctionOnMembers() {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
@@ -290,19 +293,20 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
       }
     });
     String command = "execute function --id=" + function.getId();
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionOnMembers command=" + command);
+    getLogWriter().info("testExecuteFunctionOnMembers command=" + command);
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       assertEquals(Result.Status.OK, cmdResult.getStatus());
-      LogWriterUtils.getLogWriter().info("testExecuteFunctionOnMembers cmdResult:" + cmdResult);
+      getLogWriter().info("testExecuteFunctionOnMembers cmdResult:" + cmdResult);
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testExecuteFunctionOnMembers stringResult:" + stringResult);
+      getLogWriter().info("testExecuteFunctionOnMembers stringResult:" + stringResult);
       assertTrue(stringResult.contains("Execution summary"));
     } else {
       fail("testExecuteFunctionOnMembers did not return CommandResult");
     }
   }
 
+  @Test
   public void testExecuteFunctionOnMembersWithArgs() {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
@@ -324,13 +328,13 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
 
     String command = "execute function --id=" + function.getId() + " --arguments=arg1,arg2";
 
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionOnMembersWithArgs command=" + command);
+    getLogWriter().info("testExecuteFunctionOnMembersWithArgs command=" + command);
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       assertEquals(Result.Status.OK, cmdResult.getStatus());
-      LogWriterUtils.getLogWriter().info("testExecuteFunctionOnMembersWithArgs cmdResult:" + cmdResult);
+      getLogWriter().info("testExecuteFunctionOnMembersWithArgs cmdResult:" + cmdResult);
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testExecuteFunctionOnMembersWithArgs stringResult:" + stringResult);
+      getLogWriter().info("testExecuteFunctionOnMembersWithArgs stringResult:" + stringResult);
       assertTrue(stringResult.contains("Execution summary"));
       assertTrue(stringResult.contains("arg1"));
     } else {
@@ -338,6 +342,7 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testExecuteFunctionOnGroups() {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
@@ -386,17 +391,17 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
     });
 
     String command = "execute function --id=" + TestFunction.TEST_FUNCTION1 + " --groups=Group1,Group2";
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionOnGroups command=" + command);
+    getLogWriter().info("testExecuteFunctionOnGroups command=" + command);
     CommandResult cmdResult = executeCommand(command);
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionOnGroups cmdResult=" + cmdResult);
+    getLogWriter().info("testExecuteFunctionOnGroups cmdResult=" + cmdResult);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
     TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
     List<String> members = resultData.retrieveAllValues("Member ID/Name");
-    LogWriterUtils.getLogWriter().info("testExecuteFunctionOnGroups members=" + members);
+    getLogWriter().info("testExecuteFunctionOnGroups members=" + members);
     assertTrue(members.size() == 2 && members.contains(vm1id) && members.contains(vm2id));
   }
 
-
+  @Test
   public void testDestroyOnMember() {
     createDefaultSetup(null);
     Function function = new TestFunction(true, TestFunction.TEST_FUNCTION1);
@@ -404,18 +409,19 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
     final VM vm1 = Host.getHost(0).getVM(1);
     final String vm1MemberId = (String) vm1.invoke(() -> FunctionCommandsDUnitTest.getMemberId());
     String command = "destroy function --id=" + function.getId() + " --member=" + vm1MemberId;
-    LogWriterUtils.getLogWriter().info("testDestroyOnMember command=" + command);
+    getLogWriter().info("testDestroyOnMember command=" + command);
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
-      LogWriterUtils.getLogWriter().info("testDestroyOnMember strCmdResult=" + strCmdResult);
+      getLogWriter().info("testDestroyOnMember strCmdResult=" + strCmdResult);
       assertTrue(strCmdResult.contains("Destroyed TestFunction1 Successfully"));
     } else {
       fail("testDestroyOnMember failed as did not get CommandResult");
     }
   }
 
+  @Test
   public void testDestroyOnGroups() {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
@@ -462,14 +468,14 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
     });
 
     String command = "destroy function --id=" + TestFunction.TEST_FUNCTION1 + " --groups=Group1,Group2";
-    LogWriterUtils.getLogWriter().info("testDestroyOnGroups command=" + command);
+    getLogWriter().info("testDestroyOnGroups command=" + command);
     CommandResult cmdResult = executeCommand(command);
-    LogWriterUtils.getLogWriter().info("testDestroyOnGroups cmdResult=" + cmdResult);
+    getLogWriter().info("testDestroyOnGroups cmdResult=" + cmdResult);
     assertEquals(Result.Status.OK, cmdResult.getStatus());
     String content = null;
     try {
       content = cmdResult.getContent().get("message").toString();
-      LogWriterUtils.getLogWriter().info("testDestroyOnGroups content = " + content);
+      getLogWriter().info("testDestroyOnGroups content = " + content);
     } catch (GfJsonException e) {
       fail("testDestroyOnGroups exception=" + e);
     }
@@ -479,6 +485,7 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
         "[\"Destroyed " + TestFunction.TEST_FUNCTION1 + " Successfully on " + vm2id + "," + vm1id + "\"]"));
   }
 
+  @Test
   public void testListFunction() {
     // Create the default setup, putting the Manager VM into Group1
     Properties localProps = new Properties();


[31/32] incubator-geode git commit: Merge branch 'develop' into feature/GEODE-17-2

Posted by ji...@apache.org.
Merge branch 'develop' into feature/GEODE-17-2

# Conflicts:
#	geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
#	geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java
#	geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
#	geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
#	geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
#	geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewayReceiverDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStartDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStopDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStartDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStopDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandListDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
#	geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 31c6054fd1c9f9436b6764d111f50fd001b88791
Parents: 2322827 f702bcf
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu Apr 14 20:05:23 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Apr 14 20:05:23 2016 -0700

----------------------------------------------------------------------
 .../apache/geode/gradle/PasswordDialog.groovy   |  41 ++
 .../LauncherLifecycleCommandsDUnitTest.java     | 132 ++---
 .../SharedConfigurationEndToEndDUnitTest.java   |   9 +-
 geode-core/build.gradle                         |   8 +-
 .../client/internal/CloseConnectionOp.java      |   0
 .../cache/client/internal/ConnectionImpl.java   |   8 +-
 .../internal/membership/gms/Services.java       |  23 +-
 .../membership/gms/fd/GMSHealthMonitor.java     |  20 +-
 .../gms/messenger/JGroupsMessenger.java         |   2 +-
 .../membership/gms/messenger/StatRecorder.java  |  30 +-
 .../gemfire/internal/SocketCreator.java         |  25 +-
 .../gemfire/internal/SocketIOWithTimeout.java   | 491 -------------------
 .../gemfire/internal/SocketInputStream.java     | 181 -------
 .../gemfire/internal/SocketInputWrapper.java    |  93 ----
 .../gemfire/internal/SocketOutputStream.java    | 174 -------
 .../gemstone/gemfire/internal/SocketUtils.java  | 220 ---------
 .../gemfire/internal/cache/EntryEventImpl.java  |   4 +-
 .../internal/cache/GemFireCacheImpl.java        |  14 +-
 .../cache/SearchLoadAndWriteProcessor.java      |  28 +-
 .../cache/execute/util/CommitFunction.java      | 141 ------
 .../execute/util/NestedTransactionFunction.java | 116 -----
 .../cache/execute/util/RollbackFunction.java    | 136 -----
 .../cache/tier/sockets/AcceptorImpl.java        |   3 +-
 .../cache/tier/sockets/CacheClientNotifier.java | 105 +---
 .../cache/tier/sockets/CacheClientUpdater.java  |   5 +-
 .../internal/cache/tier/sockets/HandShake.java  |  13 +-
 .../internal/cache/tier/sockets/Message.java    |   3 +-
 .../cache/tier/sockets/ServerConnection.java    |   5 +-
 .../tier/sockets/ServerHandShakeProcessor.java  |  26 +-
 .../tier/sockets/command/CloseConnection.java   |   0
 .../serial/SerialSecondaryGatewayListener.java  |  19 +-
 .../gemfire/internal/lang/ThrowableUtils.java   | 101 ++++
 .../gemfire/internal/shared/NativeCalls.java    |   2 +-
 .../gemfire/internal/tcp/Connection.java        |   5 +-
 .../gemfire/management/internal/RestAgent.java  |  51 +-
 .../query/dunit/SelectStarQueryDUnitTest.java   | 216 ++++----
 .../gemfire/cache30/MultiVMRegionTestCase.java  |  13 +-
 .../distributed/DistributedSystemDUnitTest.java |  20 +-
 .../gms/membership/StatRecorderJUnitTest.java   |  67 ++-
 .../gemstone/gemfire/internal/ClassBuilder.java |  13 +-
 .../gemfire/internal/JSSESocketJUnitTest.java   |  11 +-
 .../internal/JarClassLoaderJUnitTest.java       |  17 +
 .../cache/ClientServerTransactionDUnitTest.java |   2 -
 .../gemfire/internal/cache/CommitFunction.java  | 142 ++++++
 .../internal/cache/EntryEventImplTest.java      |  71 +++
 .../cache/NestedTransactionFunction.java        | 116 +++++
 .../internal/cache/RollbackFunction.java        | 137 ++++++
 .../cache/SearchLoadAndWriteProcessorTest.java  |  68 +++
 .../cache/partitioned/ShutdownAllDUnitTest.java |  10 +-
 .../PersistentRecoveryOrderDUnitTest.java       |   6 +-
 .../internal/lang/ThrowableUtilsTest.java       | 242 +++++++++
 .../cli/commands/CliCommandTestBase.java        |  34 +-
 .../cli/commands/ConfigCommandsDUnitTest.java   |  20 +
 ...eateAlterDestroyRegionCommandsDUnitTest.java |  24 +
 .../cli/commands/DeployCommandsDUnitTest.java   |   3 +
 .../commands/DiskStoreCommandsDUnitTest.java    |  17 +
 .../commands/DiskStoreCommandsJUnitTest.java    |  32 +-
 .../cli/commands/FunctionCommandsDUnitTest.java |  10 +
 .../cli/commands/IndexCommandsDUnitTest.java    |  16 +-
 .../ListAndDescribeRegionDUnitTest.java         |   8 +
 .../MiscellaneousCommandsDUnitTest.java         |  14 +
 ...laneousCommandsExportLogsPart1DUnitTest.java |  11 +
 ...laneousCommandsExportLogsPart2DUnitTest.java |  11 +
 ...laneousCommandsExportLogsPart3DUnitTest.java |  12 +
 ...laneousCommandsExportLogsPart4DUnitTest.java |  11 +
 .../cli/commands/QueueCommandsDUnitTest.java    |  15 +
 .../SharedConfigurationCommandsDUnitTest.java   |  15 +
 .../cli/commands/ShellCommandsDUnitTest.java    |   9 +
 .../cli/commands/ShowMetricsDUnitTest.java      |  13 +
 .../cli/commands/ShowStackTraceDUnitTest.java   |  10 +
 .../cli/commands/UserCommandsDUnitTest.java     |  10 +
 .../com/gemstone/gemfire/test/dunit/VM.java     |  88 ++--
 .../test/dunit/tests/BasicDUnitTest.java        |   3 +
 .../tests/GetDefaultDiskStoreNameDUnitTest.java |   1 -
 .../dunit/tests/GetTestMethodNameDUnitTest.java |   1 -
 ...ingGetPropertiesDisconnectsAllDUnitTest.java |   3 +
 ...ingGetPropertiesDisconnectsAllDUnitTest.java |   3 +
 .../cli/commands/ClientCommandsDUnitTest.java   |  46 +-
 .../DurableClientCommandsDUnitTest.java         |  19 +-
 .../internal/directory/RegionDirectory.java     |  20 +-
 .../lucene/internal/filesystem/FileSystem.java  |   5 +
 .../repository/IndexRepositoryImpl.java         |   2 +-
 .../repository/serializer/SerializerUtil.java   |  16 +-
 .../LuceneIndexRecoveryHAJUnitTest.java         |  20 +-
 .../internal/LuceneServiceImplJUnitTest.java    |  44 +-
 .../IndexRepositoryImplPerformanceTest.java     |   4 +-
 geode-site/website/Rules                        |   6 +
 geode-site/website/layouts/default.html         |   2 +-
 geode-site/website/layouts/header.html          |   2 +-
 geode-site/website/layouts/releases.html        |   1 +
 .../wan/wancommand/WANCommandTestBase.java      |   9 +
 ...anCommandCreateGatewayReceiverDUnitTest.java |  19 +-
 .../WanCommandCreateGatewaySenderDUnitTest.java |  19 +-
 ...WanCommandGatewayReceiverStartDUnitTest.java |  19 +-
 .../WanCommandGatewayReceiverStopDUnitTest.java |  19 +-
 .../WanCommandGatewaySenderStopDUnitTest.java   |  19 +-
 .../WanCommandPauseResumeDUnitTest.java         |  63 ++-
 .../wancommand/WanCommandStatusDUnitTest.java   |  19 +-
 .../ClusterConfigurationDUnitTest.java          |  41 +-
 .../web/swagger/config/RestApiPathProvider.java |  27 +-
 gradle/dependency-versions.properties           |   2 +-
 gradle/publish.gradle                           |  35 ++
 102 files changed, 1974 insertions(+), 2283 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
----------------------------------------------------------------------
diff --cc geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
index efbcf94,b394063..b5fa956
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
@@@ -45,43 -82,7 +82,10 @@@ import com.gemstone.gemfire.management.
  import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
  import com.gemstone.gemfire.test.dunit.WaitCriterion;
  import com.gemstone.gemfire.test.junit.categories.DistributedTest;
- import org.junit.FixMethodOrder;
- import org.junit.Ignore;
- import org.junit.Test;
- import org.junit.experimental.categories.Category;
- import org.junit.runners.MethodSorters;
- 
- import javax.management.MBeanServerConnection;
- import javax.management.ObjectName;
- import javax.management.Query;
- import javax.management.QueryExp;
- import javax.management.remote.JMXConnector;
- import javax.management.remote.JMXConnectorFactory;
- import javax.management.remote.JMXServiceURL;
- import java.io.BufferedReader;
- import java.io.BufferedWriter;
- import java.io.File;
- import java.io.FileFilter;
- import java.io.FileOutputStream;
- import java.io.FileReader;
- import java.io.FileWriter;
- import java.io.IOException;
- import java.io.OutputStreamWriter;
- import java.lang.management.ManagementFactory;
- import java.net.InetAddress;
- import java.nio.charset.Charset;
- import java.text.DateFormat;
- import java.text.MessageFormat;
- import java.text.SimpleDateFormat;
- import java.util.Calendar;
- import java.util.Queue;
- import java.util.Set;
- import java.util.concurrent.ConcurrentLinkedDeque;
- import java.util.concurrent.TimeUnit;
  
 +import static com.gemstone.gemfire.test.dunit.Assert.*;
 +import static com.gemstone.gemfire.test.dunit.Wait.waitForCriterion;
 +
  /**
   * The LauncherLifecycleCommandsDUnitTest class is a test suite of integration tests testing the contract and
   * functionality of the GemFire launcher lifecycle commands inside Gfsh.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
index cdf19b5,6a761b4..7d65ce7
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
@@@ -236,31 -215,12 +251,25 @@@ public abstract class CliCommandTestBas
     * @param shell   Shell to connect
     */
    protected void shellConnect(final String host, final int jmxPort, final int httpPort, HeadlessGfsh shell) {
 -    assert (host != null);
 -    assert (shell != null);
 +    assertTrue(host != null);
 +    assertTrue(shell != null);
  
-     CommandResult result = connect(host, jmxPort, httpPort, shell);
-     if (!shell.isConnectedAndReady()) {
-       throw new AssertionError(
-           "Connect command failed to connect to manager, result=" + commandResultToString(result));
-     }
-     info("Successfully connected to managing node using " + (useHttpOnConnect ? "HTTP" : "JMX"));
-     assertEquals(true, shell.isConnectedAndReady());
++    connect(host, jmxPort, httpPort, shell);
 +  }
 +
 +  protected CommandResult shellConnect(){
 +    return connect(this.jmxHost, this.jmxPort, this.httpPort, getDefaultShell());
 +  }
 +
 +  protected CommandResult connect(final String host, final int jmxPort, final int httpPort, HeadlessGfsh shell){
      final CommandStringBuilder command = new CommandStringBuilder(CliStrings.CONNECT);
 -    String endpoint;
  
 +    if(enableAuth) {
 +      command.addOption(CliStrings.CONNECT__USERNAME, username);
 +      command.addOption(CliStrings.CONNECT__PASSWORD, password);
 +    }
 +
 +    String endpoint;
      if (useHttpOnConnect) {
        endpoint = "http://" + host + ":" + httpPort + "/gemfire/v1";
        command.addOption(CliStrings.CONNECT__USE_HTTP, Boolean.TRUE.toString());
@@@ -269,9 -229,16 +278,18 @@@
        endpoint = host + "[" + jmxPort + "]";
        command.addOption(CliStrings.CONNECT__JMX_MANAGER, endpoint);
      }
 +    System.out.println(getClass().getSimpleName()+" using endpoint: "+endpoint);
  
-     return executeCommand(shell, command.toString());
+     CommandResult result = executeCommand(shell, command.toString());
+ 
+     if (!shell.isConnectedAndReady()) {
 -      throw new AssertionError(
++      throw new TestException(
+           "Connect command failed to connect to manager " + endpoint + " result=" + commandResultToString(result));
+     }
+ 
+     info("Successfully connected to managing node using " + (useHttpOnConnect ? "HTTP" : "JMX"));
+     assertEquals(true, shell.isConnectedAndReady());
++    return result;
    }
  
    /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
index b81b080,fee3978..a314681
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
@@@ -36,18 -47,6 +36,19 @@@ import com.gemstone.gemfire.test.dunit.
  import com.gemstone.gemfire.test.dunit.Wait;
  import com.gemstone.gemfire.test.dunit.WaitCriterion;
  import com.gemstone.gemfire.test.junit.categories.DistributedTest;
++import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 +import org.junit.Test;
 +import org.junit.experimental.categories.Category;
 +import org.junit.runner.RunWith;
 +import org.junit.runners.Parameterized;
 +
 +import java.io.File;
 +import java.io.FilenameFilter;
 +import java.io.IOException;
 +import java.util.Properties;
 +import java.util.regex.Pattern;
 +
 +import static com.gemstone.gemfire.test.dunit.Assert.*;
  
  /**
   * Unit tests for the DeployCommands class
@@@ -56,7 -55,6 +57,9 @@@
   */
  @Category(DistributedTest.class)
  @SuppressWarnings("serial")
++@Category(DistributedTest.class)
++@SuppressWarnings("serial")
 +@RunWith(Parameterized.class)
  public class DeployCommandsDUnitTest extends CliCommandTestBase {
  
    File newDeployableJarFile = new File("DeployCommandsDUnit1.jar");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java
index a6f616c,788e9eb..775c8d0
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommandsJUnitTest.java
@@@ -30,23 -47,7 +47,8 @@@ import com.gemstone.gemfire.management.
  import com.gemstone.gemfire.management.internal.cli.util.DiskStoreNotFoundException;
  import com.gemstone.gemfire.management.internal.cli.util.MemberNotFoundException;
  import com.gemstone.gemfire.test.junit.categories.UnitTest;
- import org.jmock.Expectations;
- import org.jmock.Mockery;
- import org.jmock.lib.legacy.ClassImposteriser;
- import org.junit.After;
- import org.junit.Assert;
- import org.junit.Before;
- import org.junit.Test;
- import org.junit.experimental.categories.Category;
- 
- import java.util.ArrayList;
- import java.util.Arrays;
- import java.util.Collections;
- import java.util.List;
- import java.util.Set;
  
- import static org.junit.Assert.*;
 +
  /**
   * The DiskStoreCommandsJUnitTest class is a test suite of test cases testing the contract and functionality of the
   * DiskStoreCommands class implementing commands in the GemFire shell (gfsh) that access and modify disk stores in

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
index e2c8e2e,123c3bc..ab8097c
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
@@@ -139,7 -133,7 +139,6 @@@ public class IndexCommandsDUnitTest ext
      return regionFactory.create(regionName);
    }
  
--  @Test
    public void testCreateKeyIndexOnRegionWithPersistence() {
      setupSystemPersist();
  
@@@ -169,7 -163,7 +168,6 @@@
      assertTrue(Status.OK.equals(commandResult.getStatus()));
    }
  
--  @Test
    public void testCreateAndDestroyIndex() {
      setupSystem();
      /***
@@@ -211,7 -205,7 +209,6 @@@
      assertFalse(resultAsString.contains(indexName));
    }
  
--  @Test
    public void testCreateIndexMultipleIterators() {
      setupSystem();
  
@@@ -236,7 -230,7 +233,6 @@@
      assertTrue(resultAsString.contains(indexName));
    }
  
--  @Test
    public void testCreateMultipleIndexes() {
      setupSystem();
      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEFINE_INDEX);
@@@ -270,7 -264,7 +266,6 @@@
      assertTrue(resultAsString.contains(indexName));
    }
  
--  @Test
    public void testClearMultipleIndexes() {
      setupSystem();
      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEFINE_INDEX);
@@@ -304,7 -298,7 +299,6 @@@
      assertTrue(!resultAsString.contains(indexName));
    }
  
--  @Test
    public void testCreateAndDestroyIndexOnMember() {
      setupSystem();
      /***
@@@ -354,7 -348,7 +348,6 @@@
      assertFalse(resultAsString.contains(VM1Name));
    }
  
--  @Test
    public void testCreateAndDestroyIndexOnGroup() {
      setupSystem();
      /***
@@@ -422,7 -416,7 +415,6 @@@
      }
    }
  
--  @Test
    public void testCreateAndDestroyIndexWithIncorrectInput() {
      setupSystem();
      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
@@@ -541,7 -535,7 +533,6 @@@
      assertTrue(commandResult.getStatus().equals(Status.ERROR));
    }
  
--  @Test
    public void testDestroyIndexWithoutIndexName() {
      setupSystem();
      CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
@@@ -603,7 -597,7 +594,6 @@@
    /**
     * Asserts that creating and destroying indexes correctly updates the shared configuration.
     */
--  @Test
    public void testCreateDestroyUpdatesSharedConfig() {
      disconnectAllFromDS();
  

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java
index 40006d2,eb52ffc..8391ffa
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java
@@@ -72,9 -72,9 +72,9 @@@ public class VM implements Serializabl
    }
  
    /**
--   * Creates a new {@code VM} that runs on a given host with a given process 
++   * Creates a new {@code VM} that runs on a given host with a given process
     * id.
--   * 
++   *
     * TODO: change pid to reflect value from {@link ProcessUtils#identifyPid()}
     */
    public VM(final Host host, final int pid, final RemoteDUnitVMIF client) {
@@@ -99,7 -99,7 +99,7 @@@
    }
  
    /**
--   * Invokes a static zero-arg method  with an {@link Object} or {@code void} 
++   * Invokes a static zero-arg method  with an {@link Object} or {@code void}
     * return type in this {@code VM}.  If the return type of the method is
     * {@code void}, {@code null} is returned.
     *
@@@ -107,12 -107,12 +107,12 @@@
     *         The class on which to invoke the method
     * @param  methodName
     *         The name of the method to invoke
--   *         
++   *
     * @throws RMIException
     *         Wraps any underlying exception thrown while invoking the method in
     *         this VM
--   *         
--   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead        
++   *
++   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead
     */
    public Object invoke(final Class targetClass, final String methodName) {
      return invoke(targetClass, methodName, new Object[0]);
@@@ -127,8 -127,8 +127,8 @@@
     *         The class on which to invoke the method
     * @param  methodName
     *         The name of the method to invoke
--   *         
--   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead       
++   *
++   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead
     */
    public AsyncInvocation invokeAsync(final Class targetClass, final String methodName) {
      return invokeAsync(targetClass, methodName, null);
@@@ -146,12 -146,12 +146,12 @@@
     * @param  args
     *         Arguments passed to the method call (must be
     *         {@link java.io.Serializable}).
--   *         
++   *
     * @throws RMIException
     *         Wraps any underlying exception thrown while invoking the method in
     *         this {@code VM}
--   *         
--   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead        
++   *
++   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead
     */
    public Object invoke(final Class targetClass, final String methodName, final Object[] args) {
      if (!this.available) {
@@@ -164,12 -164,12 +164,12 @@@
        return result.getResult();
  
      } else {
--      throw new RMIException(this, targetClass.getName(), methodName, result.getException(), result.getStackTrace()); 
++      throw new RMIException(this, targetClass.getName(), methodName, result.getException(), result.getStackTrace());
      }
    }
  
    /**
--   * Asynchronously invokes an instance method with an {@link Object} or 
++   * Asynchronously invokes an instance method with an {@link Object} or
     * {@code void} return type in this {@code VM}.  If the return type of the
     * method is {@code void}, {@code null} is returned.
     *
@@@ -180,7 -180,7 +180,7 @@@
     * @param  args
     *         Arguments passed to the method call (must be {@link
     *         java.io.Serializable}).
--   *         
++   *
     * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead
     */
    public AsyncInvocation invokeAsync(final Object targetObject, final String methodName, final Object[] args) {
@@@ -212,7 -212,7 +212,7 @@@
     *
     * @param  runnable
     *         The {@code Runnable} to be run
--   *         
++   *
     * @see SerializableRunnable
     */
    public AsyncInvocation invokeAsync(final SerializableRunnableIF runnable) {
@@@ -220,15 -220,15 +220,15 @@@
    }
    
    /**
--   * Invokes the {@code run} method of a {@link Runnable} in this VM.  Recall 
--   * that {@code run} takes no arguments and has no return value.  The 
--   * {@code Runnable} is wrapped in a {@link NamedRunnable} having the given 
++   * Invokes the {@code run} method of a {@link Runnable} in this VM.  Recall
++   * that {@code run} takes no arguments and has no return value.  The
++   * {@code Runnable} is wrapped in a {@link NamedRunnable} having the given
     * name so it shows up in DUnit logs.
     *
     * @param  runnable
     *         The {@code Runnable} to be run
--   * @param  name 
--   *         The name of the {@code Runnable}, which will be logged in DUnit 
++   * @param  name
++   *         The name of the {@code Runnable}, which will be logged in DUnit
     *         output
     *
     * @see SerializableRunnable
@@@ -238,12 -238,12 +238,12 @@@
    }
    
    /**
--   * Invokes the {@code call} method of a {@link Callable} in this {@code VM}.  
++   * Invokes the {@code call} method of a {@link Callable} in this {@code VM}.
     *
     * @param  callable
     *         The {@code Callable} to be run
--   * @param  name 
--   *         The name of the {@code Callable}, which will be logged in dunit 
++   * @param  name
++   *         The name of the {@code Callable}, which will be logged in dunit
     *         output
     *
     * @see SerializableCallable
@@@ -270,8 -270,8 +270,8 @@@
     *
     * @param  runnable
     *         The {@code Runnable} to be run
--   * @param  name 
--   *         The name of the {@code Runnable}, which will be logged in DUnit 
++   * @param  name
++   *         The name of the {@code Runnable}, which will be logged in DUnit
     *         output
     *
     * @see SerializableRunnable
@@@ -298,8 -298,8 +298,8 @@@
     *
     * @param  callable
     *         The {@code Callable} to be run
--   * @param  name 
--   *         The name of the {@code Callable}, which will be logged in DUnit 
++   * @param  name
++   *         The name of the {@code Callable}, which will be logged in DUnit
     *         output
     *
     * @see SerializableCallable
@@@ -309,7 -309,7 +309,7 @@@
    }
    
    /**
--   * Invokes the {@code call} method of a {@link Callable} in this {@code VM}. 
++   * Invokes the {@code call} method of a {@link Callable} in this {@code VM}.
     *
     * @param  callable
     *         The {@code Callable} to be run
@@@ -335,7 -335,7 +335,7 @@@
    }
  
    /**
--   * Invokes an instance method with no arguments on an object that is 
++   * Invokes an instance method with no arguments on an object that is
     * serialized into this {@code VM}.  The return type of the method can be
     * either {@link Object} or {@code void}.  If the return type of the method
     * is {@code void}, {@code null} is returned.
@@@ -348,8 -348,8 +348,8 @@@
     * @throws RMIException
     *         Wraps any underlying exception thrown while invoking the method in
     *         this {@code VM}
--   *         
--   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead.        
++   *
++   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead.
     */
    public Object invoke(final Object targetObject, final String methodName) {
      return invoke(targetObject, methodName, new Object[0]);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
----------------------------------------------------------------------
diff --cc geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
index 6abac09,6fef000..ee823d7
--- a/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
@@@ -376,15 -377,15 +376,15 @@@ public void waitForListClientMbean2()
      
      commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId1 + "\"" ;
      
--    getLogWriter().info("testDescribeClientWithServers commandStr clientId1 ="+commandString);    
++    getLogWriter().info("testDescribeClientWithServers commandStr clientId1 ="+commandString);
      
      
      CommandResult commandResultForClient1 = executeCommand(commandString);
--    getLogWriter().info("testDescribeClientWithServers commandStr clientId1="+commandResultForClient1);    
++    getLogWriter().info("testDescribeClientWithServers commandStr clientId1="+commandResultForClient1);
      
      
      String resultAsString = commandResultToString(commandResultForClient1);
--    getLogWriter().info("testDescribeClientWithServers commandStr clientId1 ="+resultAsString);   
++    getLogWriter().info("testDescribeClientWithServers commandStr clientId1 ="+resultAsString);
      assertTrue(Status.OK.equals(commandResultForClient1.getStatus()));
      
      verifyClientStats(commandResultForClient1, serverName1);
@@@ -393,15 -394,15 +393,15 @@@
      
      commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId2 + "\"" ;
      
--    getLogWriter().info("testDescribeClientWithServers commandStr1="+commandString);    
++    getLogWriter().info("testDescribeClientWithServers commandStr1="+commandString);
      
      
      CommandResult commandResultForClient2 = executeCommand(commandString);
--    getLogWriter().info("testDescribeClientWithServers commandResult1="+commandResultForClient2);    
++    getLogWriter().info("testDescribeClientWithServers commandResult1="+commandResultForClient2);
      
      
      resultAsString = commandResultToString(commandResultForClient2);
--    getLogWriter().info("testDescribeClientWithServers resultAsString1="+resultAsString);   
++    getLogWriter().info("testDescribeClientWithServers resultAsString1="+resultAsString);
      assertTrue(Status.OK.equals(commandResultForClient2.getStatus()));
      
      verifyClientStats(commandResultForClient2, serverName2);
@@@ -461,7 -462,7 +461,7 @@@
    public void testDescribeClient() throws Exception {
      setupSystem();
      
--    getLogWriter().info("testDescribeClient clientId="+clientId);    
++    getLogWriter().info("testDescribeClient clientId="+clientId);
      assertNotNull(clientId);
      
      String commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId + "\"" ;
@@@ -479,11 -480,11 +479,11 @@@
      
      
      CommandResult commandResult = executeCommand(commandString);
--    getLogWriter().info("testDescribeClient commandResult="+commandResult);    
++    getLogWriter().info("testDescribeClient commandResult="+commandResult);
      
      
      String resultAsString = commandResultToString(commandResult);
--    getLogWriter().info("testDescribeClient resultAsString="+resultAsString);   
++    getLogWriter().info("testDescribeClient resultAsString="+resultAsString);
      assertTrue(Status.OK.equals(commandResult.getStatus()));
      
      CompositeResultData resultData = (CompositeResultData) commandResult.getResultData();
@@@ -533,7 -534,7 +533,7 @@@
      setupSystem2();
      
      String commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId + "\"" ;
--    getLogWriter().info("testDescribeClientWithServers commandStr="+commandString);    
++    getLogWriter().info("testDescribeClientWithServers commandStr="+commandString);
      
      
      final VM server1 = Host.getHost(0).getVM(1);
@@@ -547,11 -548,11 +547,11 @@@
      
      
      CommandResult commandResult = executeCommand(commandString);
--    getLogWriter().info("testDescribeClientWithServers commandResult="+commandResult);    
++    getLogWriter().info("testDescribeClientWithServers commandResult="+commandResult);
      
      
      String resultAsString = commandResultToString(commandResult);
--    getLogWriter().info("testDescribeClientWithServers resultAsString="+resultAsString);   
++    getLogWriter().info("testDescribeClientWithServers resultAsString="+resultAsString);
      assertTrue(Status.OK.equals(commandResult.getStatus()));
      
      CompositeResultData resultData = (CompositeResultData) commandResult.getResultData();
@@@ -632,11 -633,11 +632,11 @@@
      });
      
      CommandResult commandResult = executeCommand(commandString);
--    getLogWriter().info("testListClient commandResult="+commandResult);    
++    getLogWriter().info("testListClient commandResult="+commandResult);
      
      
      String resultAsString = commandResultToString(commandResult);
--    getLogWriter().info("testListClient resultAsString="+resultAsString);   
++    getLogWriter().info("testListClient resultAsString="+resultAsString);
      assertTrue(Status.OK.equals(commandResult.getStatus()));
      
      
@@@ -650,8 -651,8 +650,8 @@@
      List<String> clientNames = tableRsultData.retrieveAllValues(CliStrings.LIST_CLIENT_COLUMN_Clients);
      
      
--    getLogWriter().info("testListClients serverNames : " + serverNames);    
--    getLogWriter().info("testListClients clientNames : " + clientNames);  
++    getLogWriter().info("testListClients serverNames : " + serverNames);
++    getLogWriter().info("testListClients clientNames : " + clientNames);
      assertEquals(2, serverNames.size());
      assertEquals(2, clientNames.size());    
      assertTrue(clientNames.contains(clientIds[0]));
@@@ -716,11 -717,11 +716,11 @@@
      });
      
      CommandResult commandResult = executeCommand(commandString);
--    getLogWriter().info("testListClientForServers commandResult="+commandResult);    
++    getLogWriter().info("testListClientForServers commandResult="+commandResult);
      
      
      String resultAsString = commandResultToString(commandResult);
--    getLogWriter().info("testListClientForServers resultAsString="+resultAsString);   
++    getLogWriter().info("testListClientForServers resultAsString="+resultAsString);
      assertTrue(Status.OK.equals(commandResult.getStatus()));
      
      
@@@ -1158,7 -1159,7 +1158,7 @@@ public void waitForMixedClients()
    public void testDescribeClientForNonSubscribedClient() throws Exception {
      setUpNonSubscribedClient();
      
--    getLogWriter().info("testDescribeClientForNonSubscribedClient clientId="+clientId);    
++    getLogWriter().info("testDescribeClientForNonSubscribedClient clientId="+clientId);
      assertNotNull(clientId);
      
      String commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId + "\"" ;
@@@ -1175,11 -1176,11 +1175,11 @@@
      
      
      CommandResult commandResult = executeCommand(commandString);
--    getLogWriter().info("testDescribeClientForNonSubscribedClient commandResult="+commandResult);    
++    getLogWriter().info("testDescribeClientForNonSubscribedClient commandResult="+commandResult);
      
      
      String resultAsString = commandResultToString(commandResult);
--    getLogWriter().info("testDescribeClientForNonSubscribedClient resultAsString="+resultAsString);   
++    getLogWriter().info("testDescribeClientForNonSubscribedClient resultAsString="+resultAsString);
      assertTrue(Status.OK.equals(commandResult.getStatus()));
      
      CompositeResultData resultData = (CompositeResultData) commandResult.getResultData();
@@@ -1249,7 -1250,7 +1249,7 @@@
      executeAndVerifyResultsForMixedClients(commandString, serverName );    
      
      String commandString2 = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientIds[1] + "\"" ;
--    getLogWriter().info("testDescribeMixClientWithServers commandString2="+commandString2);   
++    getLogWriter().info("testDescribeMixClientWithServers commandString2="+commandString2);
      
      
      executeAndVerifyResultsForMixedClients(commandString2,serverName );
@@@ -1262,7 -1263,7 +1262,7 @@@
    
  void executeAndVerifyResultsForMixedClients(String commandString, String serverName){
    CommandResult commandResult = executeCommand(commandString);
--  getLogWriter().info("testDescribeMixClientWithServers commandResult="+commandResult);    
++  getLogWriter().info("testDescribeMixClientWithServers commandResult="+commandResult);
    
    
    String resultAsString = commandResultToString(commandResult);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java
----------------------------------------------------------------------
diff --cc geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java
index 1ee5c33,719fd92..49a978c
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java
@@@ -16,6 -16,16 +16,15 @@@
   */
  package com.gemstone.gemfire.internal.cache.wan.wancommand;
  
 -import static com.gemstone.gemfire.test.dunit.Assert.*;
 -
+ import java.io.File;
+ import java.io.IOException;
+ import java.util.ArrayList;
+ import java.util.List;
+ import java.util.Properties;
+ import java.util.Set;
++
+ import javax.management.remote.JMXConnectorServer;
+ 
  import com.gemstone.gemfire.cache.Cache;
  import com.gemstone.gemfire.cache.CacheFactory;
  import com.gemstone.gemfire.cache.DiskStore;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewayReceiverDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStartDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStopDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStopDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
----------------------------------------------------------------------
diff --cc geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
index 8462a37,b54b27f..5a352c1
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
@@@ -128,29 -129,29 +129,27 @@@ public class WanCommandPauseResumeDUnit
      props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
      props.setProperty(DistributionConfig.DISTRIBUTED_SYSTEM_ID_NAME, "1");
      props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + punePort + "]");
 -    createDefaultSetup(props);
 +    setUpJmxManagerOnVm0ThenConnect(props);
  
 -    Integer nyPort = (Integer) vm2.invoke(() -> WANCommandTestBase.createFirstRemoteLocator( 2, punePort ));
 +    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator( 2, punePort ));
  
-     vm3.invoke(() -> createCache( punePort ));
-     vm3.invoke(() -> createSender( "ln",
+     vm3.invoke(() -> WANCommandTestBase.createCache( punePort ));
+     vm3.invoke(() -> WANCommandTestBase.createSender( "ln",
              2, false, 100, 400, false, false, null, true ));
-     vm4.invoke(() -> createCache( punePort ));
-     vm4.invoke(() -> createSender( "ln",
+     vm4.invoke(() -> WANCommandTestBase.createCache( punePort ));
+     vm4.invoke(() -> WANCommandTestBase.createSender( "ln",
              2, false, 100, 400, false, false, null, true ));
-     vm5.invoke(() -> createCache( punePort ));
-     vm5.invoke(() -> createSender( "ln",
+     vm5.invoke(() -> WANCommandTestBase.createCache( punePort ));
+     vm5.invoke(() -> WANCommandTestBase.createSender( "ln",
              2, false, 100, 400, false, false, null, true ));
  
 -    vm3.invoke(() -> WANCommandTestBase.startSender( "ln" ));
 -    vm4.invoke(() -> WANCommandTestBase.startSender( "ln" ));
 -    vm5.invoke(() -> WANCommandTestBase.startSender( "ln" ));
 +    vm3.invoke(() -> startSender( "ln" ));
-     vm4.invoke(() -> startSender( "ln" ));
-     vm5.invoke(() -> startSender( "ln" ));
  
-     vm3.invoke(() -> verifySenderState(
+     vm3.invoke(() -> WANCommandTestBase.verifySenderState(
              "ln", true, false ));
-     vm4.invoke(() -> verifySenderState(
+     vm4.invoke(() -> WANCommandTestBase.verifySenderState(
              "ln", true, false ));
-     vm5.invoke(() -> verifySenderState(
+     vm5.invoke(() -> WANCommandTestBase.verifySenderState(
              "ln", true, false ));
  
      pause(10000);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/31c6054f/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
----------------------------------------------------------------------


[30/32] incubator-geode git commit: GEODE-1231: remove test013StartServerWithSpring

Posted by ji...@apache.org.
GEODE-1231: remove test013StartServerWithSpring


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

Branch: refs/heads/feature/GEODE-17-2
Commit: f702bcfe5d8d7546c32ac76bd86591bba708cc63
Parents: 70c03a7
Author: Kirk Lund <kl...@apache.org>
Authored: Thu Apr 14 16:13:58 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Thu Apr 14 16:27:45 2016 -0700

----------------------------------------------------------------------
 .../LauncherLifecycleCommandsDUnitTest.java     | 62 --------------------
 1 file changed, 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f702bcfe/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
index 4553140..b394063 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
@@ -813,68 +813,6 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     assertEquals(Status.NOT_RESPONDING, locatorState.getStatus());
   }
 
-  @Ignore("Disabled until GEODE-1025, SGF-476 are resolved")
-  @Test
-  public void test013StartServerWithSpring() {
-    String pathname = (getClass().getSimpleName() + "_" + getTestMethodName());
-    File workingDirectory = new File(pathname);
-
-    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
-
-    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_SERVER);
-
-    command.addOption(CliStrings.START_SERVER__NAME, getClass().getSimpleName().concat("_").concat(getTestMethodName()));
-    command.addOption(CliStrings.START_SERVER__USE_CLUSTER_CONFIGURATION, Boolean.FALSE.toString());
-    command.addOption(CliStrings.START_SERVER__LOG_LEVEL, "config");
-    command.addOption(CliStrings.START_SERVER__INCLUDE_SYSTEM_CLASSPATH);
-    command.addOption(CliStrings.START_SERVER__DISABLE_DEFAULT_SERVER);
-    command.addOption(CliStrings.START_SERVER__DIR, pathname);
-    command.addOption(CliStrings.START_SERVER__SPRING_XML_LOCATION, "spring/spring-gemfire-context.xml");
-
-    CommandResult result = executeCommand(command.toString());
-
-    assertNotNull(result);
-    assertEquals(Result.Status.OK, result.getStatus());
-
-    final ServerLauncher springGemFireServer = new ServerLauncher.Builder().setCommand(
-        ServerLauncher.Command.STATUS).setWorkingDirectory(
-        IOUtils.tryGetCanonicalPathElseGetAbsolutePath(workingDirectory)).build();
-
-    assertNotNull(springGemFireServer);
-
-    ServerState serverState = springGemFireServer.status();
-    
-    assertNotNull(serverState);
-    assertEquals(Status.ONLINE, serverState.getStatus());
-    
-    //Ensure the member name is what is set through spring
-    String logFile = serverState.getLogFile();
-    assertTrue("Log file name was not configured from spring context: " + logFile, logFile.contains("spring_server.log"));
-
-    // Now that the GemFire Server bootstrapped with Spring started up OK, stop it!
-    stopServer(springGemFireServer.getWorkingDirectory());
-
-    WaitCriterion waitCriteria = new WaitCriterion() {
-      @Override
-      public boolean done() {
-        ServerState serverState = springGemFireServer.status();
-        return (serverState != null && Status.NOT_RESPONDING.equals(serverState.getStatus()));
-      }
-
-      @Override
-      public String description() {
-        return "wait for the Locator to stop; the Locator will no longer respond after it stops";
-      }
-    };
-
-    waitForCriterion(waitCriteria, TimeUnit.SECONDS.toMillis(15), TimeUnit.SECONDS.toMillis(5), true);
-
-    serverState = springGemFireServer.status();
-
-    assertNotNull(serverState);
-    assertEquals(Status.NOT_RESPONDING, serverState.getStatus());
-  }
-
   @Test
   public void test014GemFireServerJvmProcessTerminatesOnOutOfMemoryError() throws Exception {
     int ports[] = AvailablePortHelper.getRandomAvailableTCPPorts(2);


[23/32] incubator-geode git commit: GEODE-1162: convert all CLI command DUnit tests to JUnit 4

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
index 7151ffd..1b0c9cf 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
@@ -16,6 +16,25 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.IgnoredException.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -55,35 +74,18 @@ import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData.S
 import com.gemstone.gemfire.management.internal.cli.result.ResultData;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.IgnoredException;
 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 hydra.GsRandom;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-import java.util.Set;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Dunit class for testing gemfire data commands : get, put, remove, select, rebalance
- *
  */
-
+@Category(DistributedTest.class)
 @SuppressWarnings("serial")
 public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
@@ -117,16 +119,11 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
   final static int COUNT = 5;
 
-  public GemfireDataCommandsDUnitTest(String name) {
-    super(name);
-  }
-
   public static String getMemberId() {
-    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    Cache cache = new GemfireDataCommandsDUnitTest().getCache();
     return cache.getDistributedSystem().getDistributedMember().getId();
   }
 
-
   void setupForGetPutRemoveLocateEntry(String testName) {
     final VM vm1 = Host.getHost(0).getVM(1);
     final VM vm2 = Host.getHost(0).getVM(2);
@@ -142,29 +139,29 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
         RegionFactory regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
         Region dataRegion = regionFactory.create(DATA_REGION_NAME);
         assertNotNull(dataRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
 
         dataRegion = dataRegion.createSubregion(DATA_REGION_NAME_CHILD_1, dataRegion.getAttributes());
         assertNotNull(dataRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
 
         dataRegion = dataRegion.createSubregion(DATA_REGION_NAME_CHILD_1_2, dataRegion.getAttributes());
         assertNotNull(dataRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
 
         dataRegion = regionFactory.create(DATA_REGION_NAME_VM1);
         assertNotNull(dataRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
 
         PartitionAttributes partitionAttrs = new PartitionAttributesFactory().setRedundantCopies(2).create();
         RegionFactory<Object, Object> partitionRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
         partitionRegionFactory.setPartitionAttributes(partitionAttrs);
         Region dataParRegion = partitionRegionFactory.create(DATA_PAR_REGION_NAME);
         assertNotNull(dataParRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataParRegion);
+        getLogWriter().info("Created Region " + dataParRegion);
         dataParRegion = partitionRegionFactory.create(DATA_PAR_REGION_NAME_VM1);
         assertNotNull(dataParRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataParRegion);
+        getLogWriter().info("Created Region " + dataParRegion);
 
       }
     });
@@ -175,19 +172,19 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
         RegionFactory regionFactory = cache.createRegionFactory(RegionShortcut.REPLICATE);
         Region dataRegion = regionFactory.create(DATA_REGION_NAME);
         assertNotNull(dataRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
 
         dataRegion = dataRegion.createSubregion(DATA_REGION_NAME_CHILD_1, dataRegion.getAttributes());
         assertNotNull(dataRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
 
         dataRegion = dataRegion.createSubregion(DATA_REGION_NAME_CHILD_1_2, dataRegion.getAttributes());
         assertNotNull(dataRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
 
         dataRegion = regionFactory.create(DATA_REGION_NAME_VM2);
         assertNotNull(dataRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataRegion);
+        getLogWriter().info("Created Region " + dataRegion);
 
 
         PartitionAttributes partitionAttrs = new PartitionAttributesFactory().setRedundantCopies(2).create();
@@ -195,18 +192,18 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
         partitionRegionFactory.setPartitionAttributes(partitionAttrs);
         Region dataParRegion = partitionRegionFactory.create(DATA_PAR_REGION_NAME);
         assertNotNull(dataParRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataParRegion);
+        getLogWriter().info("Created Region " + dataParRegion);
         dataParRegion = partitionRegionFactory.create(DATA_PAR_REGION_NAME_VM2);
         assertNotNull(dataParRegion);
-        LogWriterUtils.getLogWriter().info("Created Region " + dataParRegion);
+        getLogWriter().info("Created Region " + dataParRegion);
 
       }
     });
 
     final String vm1MemberId = (String) vm1.invoke(() -> GemfireDataCommandsDUnitTest.getMemberId());
     final String vm2MemberId = (String) vm2.invoke(() -> GemfireDataCommandsDUnitTest.getMemberId());
-    LogWriterUtils.getLogWriter().info("Vm1 ID : " + vm1MemberId);
-    LogWriterUtils.getLogWriter().info("Vm2 ID : " + vm2MemberId);
+    getLogWriter().info("Vm1 ID : " + vm1MemberId);
+    getLogWriter().info("Vm2 ID : " + vm2MemberId);
 
     final VM manager = Host.getHost(0).getVM(0);
 
@@ -222,10 +219,10 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
             ManagerMXBean bean1 = service.getManagerMXBean();
             DistributedRegionMXBean bean2 = service.getDistributedRegionMXBean(DATA_REGION_NAME_PATH);
             if (bean1 == null) {
-              LogWriterUtils.getLogWriter().info("Still probing for ManagerMBean");
+              getLogWriter().info("Still probing for ManagerMBean");
               return false;
             } else {
-              LogWriterUtils.getLogWriter().info("Still probing for DistributedRegionMXBean=" + bean2);
+              getLogWriter().info("Still probing for DistributedRegionMXBean=" + bean2);
               return (bean2 != null);
             }
           }
@@ -236,7 +233,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
           }
         };
 
-        Wait.waitForCriterion(waitForMaangerMBean, 30000, 2000, true);
+        waitForCriterion(waitForMaangerMBean, 30000, 2000, true);
 
         assertNotNull(service.getMemberMXBean());
         assertNotNull(service.getManagerMXBean());
@@ -266,7 +263,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
             }
 
             if (!flag) {
-              LogWriterUtils.getLogWriter().info(
+              getLogWriter().info(
                   "Still probing for regionMbeans " + DATA_REGION_NAME_PATH + "=" + beans[0] + " " + DATA_REGION_NAME_VM1_PATH + "=" + beans[1] + " " + DATA_REGION_NAME_VM2_PATH + "=" + beans[2] + " " + DATA_PAR_REGION_NAME_PATH + "=" + beans[3] + " " + DATA_PAR_REGION_NAME_VM1_PATH + "=" + beans[4] + " " + DATA_PAR_REGION_NAME_VM2_PATH + "=" + beans[5] + " "
                   //+ DATA_REGION_NAME_CHILD_1_PATH
                   // +"="+ beans[6]  + " " + DATA_REGION_NAME_CHILD_1_2_PATH
@@ -274,7 +271,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
               );
               return false;
             } else {
-              LogWriterUtils.getLogWriter().info(
+              getLogWriter().info(
                   "Probing complete for regionMbeans " + DATA_REGION_NAME_PATH + "=" + beans[0] + " " + DATA_REGION_NAME_VM1_PATH + "=" + beans[1] + " " + DATA_REGION_NAME_VM2_PATH + "=" + beans[2] + " " + DATA_PAR_REGION_NAME_PATH + "=" + beans[3] + " " + DATA_PAR_REGION_NAME_VM1_PATH + "=" + beans[4] + " " + DATA_PAR_REGION_NAME_VM2_PATH + "=" + beans[5] + " "
                   //+ DATA_REGION_NAME_CHILD_1_PATH
                   // +"="+ beans[6]  + " " + DATA_REGION_NAME_CHILD_1_2_PATH
@@ -296,7 +293,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
           }
         };
 
-        Wait.waitForCriterion(waitForRegionMBeans, 30000, 2000, true);
+        waitForCriterion(waitForRegionMBeans, 30000, 2000, true);
 
         String regions[] = {DATA_REGION_NAME_PATH, DATA_REGION_NAME_VM1_PATH, DATA_REGION_NAME_VM2_PATH, DATA_PAR_REGION_NAME_PATH, DATA_PAR_REGION_NAME_VM1_PATH, DATA_PAR_REGION_NAME_VM2_PATH, /*DATA_REGION_NAME_CHILD_1_PATH, DATA_REGION_NAME_CHILD_1_2_PATH*/};
 
@@ -304,7 +301,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
           bean = service.getDistributedRegionMXBean(region);
           assertNotNull(bean);
           String[] membersName = bean.getMembers();
-          LogWriterUtils.getLogWriter().info(
+          getLogWriter().info(
               "Members Array for region " + region + " : " + StringUtils.objectToString(membersName, true, 10));
           if (bean.getMemberCount() < 1) fail(
               "Even after waiting mbean reports number of member hosting region " + DATA_REGION_NAME_VM1_PATH + " is less than one");
@@ -369,7 +366,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       query = query.replace("?" + i, r);
       i++;
     }
-    LogWriterUtils.getLogWriter().info("Checking members for query : " + query);
+    getLogWriter().info("Checking members for query : " + query);
     QCompiler compiler = new QCompiler();
     Set<String> regionsInQuery = null;
     try {
@@ -377,11 +374,11 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       Set regionSet = new HashSet();
       compiledQuery.getRegionsInQuery(regionSet, null);//GFSH ENV VARIBLES
       regionsInQuery = Collections.unmodifiableSet(regionSet);
-      LogWriterUtils.getLogWriter().info("Region in query : " + regionsInQuery);
+      getLogWriter().info("Region in query : " + regionsInQuery);
       if (regionsInQuery.size() > 0) {
         Set<DistributedMember> members = DataCommands.getQueryRegionsAssociatedMembers(regionsInQuery, cache,
             returnAll);
-        LogWriterUtils.getLogWriter().info("Members for Region in query : " + members);
+        getLogWriter().info("Members for Region in query : " + members);
         if (expectedMembers != -1) {
           assertNotNull(members);
           assertEquals(expectedMembers, members.size());
@@ -390,7 +387,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
         assertEquals(-1, expectedMembers);//Regions do not exist at all
       }
     } catch (QueryInvalidException qe) {
-      Assert.fail("Invalid Query", qe);
+      fail("Invalid Query", qe);
     }
   }
 
@@ -453,7 +450,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
         MemberMXBean member = service.getMemberMXBean();
         String cmdResult = member.processCommand(query);
         assertNotNull(cmdResult);
-        LogWriterUtils.getLogWriter().info("Text Command Output : " + cmdResult);
+        getLogWriter().info("Text Command Output : " + cmdResult);
       }
     });
   }
@@ -495,17 +492,17 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     validateSelectResult(cmdResult, true, (randomInteger + 1), expectedCols);
   }
 
-  public void doTestSelectWithGfshEnvVariables() {
+  public void doTestSelectWithGfshEnvVariables(boolean statusActive) {
     Random random = new Random(System.nanoTime());
     int randomInteger = random.nextInt(COUNT);
     String query = "query --query=\"select ID , status , createTime , pk, floatMinValue from ${DATA_REGION} where ID <= ${PORTFOLIO_ID}" + " and status='${STATUS}'" + "\" --interactive=false";
     executeCommand("set variable --name=DATA_REGION --value=" + DATA_REGION_NAME_PATH);
     executeCommand("set variable --name=PORTFOLIO_ID --value=" + randomInteger);
-    executeCommand("set variable --name=STATUS --value=" + (new GsRandom().nextBoolean() ? "active" : "inactive"));
+    executeCommand("set variable --name=STATUS --value=" + (statusActive ? "active" : "inactive"));
     CommandResult cmdResult = executeCommand(query);
     printCommandOutput(cmdResult);
     validateSelectResult(cmdResult, true, -1, null);
-    IgnoredException ex = IgnoredException.addIgnoredException(QueryInvalidException.class.getSimpleName(), Host.getHost(0).getVM(0));
+    IgnoredException ex = addIgnoredException(QueryInvalidException.class.getSimpleName(), Host.getHost(0).getVM(0));
     try {
       query = "query --query=\"select ID , status , createTime , pk, floatMinValue from ${DATA_REGION2} where ID <= ${PORTFOLIO_ID2}" + " and status='${STATUS2}'" + "\" --interactive=false";
       cmdResult = executeCommand(query);
@@ -523,11 +520,12 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     validateResult(cmdResult, true);
   }
 
-
+  @Test
   public void testSelectCommand() {
     setupForSelect();
     doTestGetRegionAssociatedMembersForSelect();
-    doTestSelectWithGfshEnvVariables();
+    doTestSelectWithGfshEnvVariables(true);
+    doTestSelectWithGfshEnvVariables(false);
     doTestSelectProjection();
     doTestBug48013();
     doTestSelectProjectionProcessCommand();
@@ -536,7 +534,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     doTestSelectBeansWithNestedFieldAsResult();
   }
 
-
+  @Test
   public void testPrimitivesWithDataCommands() {
     setupForGetPutRemoveLocateEntry("testPrimitives");
     Byte byteKey = Byte.parseByte("41");
@@ -550,15 +548,15 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     Double doubleKey = Double.valueOf("12432.235425");
     Double doubleValue = Double.valueOf("111111.111111");
 
-    LogWriterUtils.getLogWriter().info("Testing Byte Wrappers");
+    getLogWriter().info("Testing Byte Wrappers");
     testGetPutLocateEntryFromShellAndGemfire(byteKey, byteValue, Byte.class, true, true);
-    LogWriterUtils.getLogWriter().info("Testing Short Wrappers");
+    getLogWriter().info("Testing Short Wrappers");
     testGetPutLocateEntryFromShellAndGemfire(shortKey, shortValue, Short.class, true, true);
-    LogWriterUtils.getLogWriter().info("Testing Integer Wrappers");
+    getLogWriter().info("Testing Integer Wrappers");
     testGetPutLocateEntryFromShellAndGemfire(integerKey, integerValue, Integer.class, true, true);
-    LogWriterUtils.getLogWriter().info("Testing Float Wrappers");
+    getLogWriter().info("Testing Float Wrappers");
     testGetPutLocateEntryFromShellAndGemfire(floatKey, flaotValue, Float.class, true, true);
-    LogWriterUtils.getLogWriter().info("Testing Double Wrappers");
+    getLogWriter().info("Testing Double Wrappers");
     testGetPutLocateEntryFromShellAndGemfire(doubleKey, doubleValue, Double.class, true, true);
   }
 
@@ -683,6 +681,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testSimplePutCommand() {
     final String keyPrefix = "testKey";
     final String valuePrefix = "testValue";
@@ -760,7 +759,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     try {
       for (String col : expectedCols) {
         boolean found = false;
-        LogWriterUtils.getLogWriter().info("Validating column " + col);
+        getLogWriter().info("Validating column " + col);
         for (int i = 0; i < array.size(); i++) {
           String header = (String) array.get(i);
           if (col.equals(header)) found = true;
@@ -768,7 +767,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
         assertEquals(true, found);
       }
     } catch (GfJsonException e) {
-      Assert.fail("Error accessing table data", e);
+      fail("Error accessing table data", e);
     }
   }
 
@@ -781,7 +780,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     try {
       for (String col : expectedCols) {
         boolean found = false;
-        LogWriterUtils.getLogWriter().info("Validating column " + col);
+        getLogWriter().info("Validating column " + col);
         for (int i = 0; i < array.size(); i++) {
           String header = (String) array.get(i);
           if (col.equals(header)) found = true;
@@ -795,7 +794,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
       }
     } catch (GfJsonException e) {
-      Assert.fail("Error accessing table data", e);
+      fail("Error accessing table data", e);
     }
   }
 
@@ -815,7 +814,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
             assertEquals(cols.length, array.size());
             for (String col : cols) {
               boolean found = false;
-              LogWriterUtils.getLogWriter().info("Validating column " + col);
+              getLogWriter().info("Validating column " + col);
               for (int i = 0; i < array.size(); i++) {
                 String header = (String) array.get(i);
                 if (col.equals(header)) found = true;
@@ -823,14 +822,14 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
               assertEquals(true, found);
             }
           } catch (GfJsonException e) {
-            Assert.fail("Error accessing table data", e);
+            fail("Error accessing table data", e);
           }
         }
       }
     } else fail("Expected CompositeResult Returned Result Type " + cmdResult.getType());
   }
 
-
+  @Test
   public void testSimplePutIfAbsentCommand() {
     final String keyPrefix = "testKey";
     final String valuePrefix = "testValue";
@@ -892,6 +891,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
   }
 
+  @Test
   public void testSimpleRemoveCommand() {
     final String keyPrefix = "testKey";
     final String valuePrefix = "testValue";
@@ -948,6 +948,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     vm2.invoke(checkRemoveKeys);
   }
 
+  @Test
   public void testSimpleGetLocateEntryCommand() {
     final String keyPrefix = "testKey";
     final String valuePrefix = "testValue";
@@ -994,6 +995,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
   }
 
+  @Test
   public void testRecursiveLocateEntryCommand() {
     final String keyPrefix = "testKey";
     final String valuePrefix = "testValue";
@@ -1036,6 +1038,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
   }
 
+  @Test
   public void testGetLocateEntryFromRegionOnDifferentVM() {
     final String keyPrefix = "testKey";
     final String valuePrefix = "testValue";
@@ -1116,6 +1119,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testGetLocateEntryLocationsForPR() {
     final String keyPrefix = "testKey";
     final String valuePrefix = "testValue";
@@ -1151,6 +1155,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testPutFromRegionOnDifferentVM() {
     final String keyPrefix = "testKey";
     final String valuePrefix = "testValue";
@@ -1207,6 +1212,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     vm2.invoke(checkPutKeysInVM2);
   }
 
+  @Test
   public void testGetLocateEntryJsonKeys() {
     final String keyPrefix = "testKey";
 
@@ -1277,7 +1283,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       String valueJson = valueTemplate.replaceAll("\\?1", population);
       valueJson = valueJson.replaceAll("\\?2", area);
       valueJson = valueJson.replaceAll("\\?", keyString);
-      LogWriterUtils.getLogWriter().info("Getting key with json key : " + keyJson);
+      getLogWriter().info("Getting key with json key : " + keyJson);
       command = command + " " + "--key=" + keyJson + " --region=" + DATA_REGION_NAME_PATH + " --key-class=" + Key1.class.getCanonicalName();
       command = command + " --value-class=" + Value2.class.getCanonicalName();
       CommandResult cmdResult = executeCommand(command);
@@ -1306,7 +1312,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       String valueJson = valueTemplate.replaceAll("\\?1", population);
       valueJson = valueJson.replaceAll("\\?2", area);
       valueJson = valueJson.replaceAll("\\?", keyString);
-      LogWriterUtils.getLogWriter().info("Getting key with json key : " + keyJson);
+      getLogWriter().info("Getting key with json key : " + keyJson);
       command = command + " " + "--key=" + keyJson + " --region=" + DATA_REGION_NAME_PATH + " --key-class=" + Key1.class.getCanonicalName();
       command = command + " --value-class=" + Value2.class.getCanonicalName();
       CommandResult cmdResult = executeCommand(command);
@@ -1326,6 +1332,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testPutJsonKeys() {
     final String keyPrefix = "testKey";
 
@@ -1343,8 +1350,8 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       String valueJson = valueTemplate.replaceAll("\\?1", population);
       valueJson = valueJson.replaceAll("\\?2", area);
       valueJson = valueJson.replaceAll("\\?", keyString);
-      LogWriterUtils.getLogWriter().info("Putting key with json key : " + keyJson);
-      LogWriterUtils.getLogWriter().info("Putting key with json valye : " + valueJson);
+      getLogWriter().info("Putting key with json key : " + keyJson);
+      getLogWriter().info("Putting key with json valye : " + valueJson);
       command = command + " " + "--key=" + keyJson + " --value=" + valueJson + " --region=" + DATA_REGION_NAME_PATH;
       command = command + " --key-class=" + Key1.class.getCanonicalName() + " --value-class=" + Value2.class.getCanonicalName();
       ;
@@ -1372,8 +1379,8 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       valueJson = valueJson.replaceAll("\\?set", set);
       valueJson = valueJson.replaceAll("\\?map", map);
 
-      LogWriterUtils.getLogWriter().info("Putting key with json key : " + keyJson);
-      LogWriterUtils.getLogWriter().info("Putting key with json valye : " + valueJson);
+      getLogWriter().info("Putting key with json key : " + keyJson);
+      getLogWriter().info("Putting key with json valye : " + valueJson);
       command = command + " " + "--key=" + keyJson + " --value=" + valueJson + " --region=" + DATA_REGION_NAME_PATH;
       command = command + " --key-class=" + Key1.class.getCanonicalName() + " --value-class=" + Car.class.getCanonicalName();
       ;
@@ -1447,6 +1454,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     validateResult(cmdResult, false);
   }
 
+  @Test
   public void testRemoveJsonCommand() {
     final String keyPrefix = "testKey";
 
@@ -1483,7 +1491,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       String command = "remove";
       String keyString = keyPrefix + i;
       String keyJson = keyTemplate.replaceAll("\\?", keyString);
-      LogWriterUtils.getLogWriter().info("Removing key with json key : " + keyJson);
+      getLogWriter().info("Removing key with json key : " + keyJson);
       command = command + " " + "--key=" + keyJson + " --region=" + DATA_REGION_NAME_PATH + " --key-class=" + Key1.class.getCanonicalName();
       CommandResult cmdResult = executeCommand(command);
       printCommandOutput(cmdResult);
@@ -1524,6 +1532,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     return regionFactory.create(regionName);
   }
 
+  @Test
   public void testImportExportData() throws InterruptedException, IOException {
     final String regionName = "Region1";
     final String exportFileName = "export.gfd";
@@ -1564,8 +1573,8 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       CommandResult cmdResult = executeCommand(commandString);
       String resultAsString = commandResultToString(cmdResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
-      LogWriterUtils.getLogWriter().info("Command Output");
-      LogWriterUtils.getLogWriter().info(resultAsString);
+      getLogWriter().info("Command Output");
+      getLogWriter().info(resultAsString);
 
       vm1.invoke(new SerializableRunnable() {
         public void run() {
@@ -1588,8 +1597,8 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       cmdResult = executeCommand(commandString);
       resultAsString = commandResultToString(cmdResult);
 
-      LogWriterUtils.getLogWriter().info("Result of import data");
-      LogWriterUtils.getLogWriter().info(resultAsString);
+      getLogWriter().info("Result of import data");
+      getLogWriter().info(resultAsString);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
       /**
@@ -1614,8 +1623,8 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
       cmdResult = executeCommand(commandString);
       resultAsString = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("Result of import data with wrong region name");
-      LogWriterUtils.getLogWriter().info(resultAsString);
+      getLogWriter().info("Result of import data with wrong region name");
+      getLogWriter().info(resultAsString);
       assertEquals(Result.Status.ERROR, cmdResult.getStatus());
 
       csb = new CommandStringBuilder(CliStrings.IMPORT_DATA);
@@ -1626,8 +1635,8 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
       cmdResult = executeCommand(commandString);
       resultAsString = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("Result of import data with wrong file");
-      LogWriterUtils.getLogWriter().info(resultAsString);
+      getLogWriter().info("Result of import data with wrong file");
+      getLogWriter().info(resultAsString);
       assertEquals(Result.Status.ERROR, cmdResult.getStatus());
 
     } finally {
@@ -1687,7 +1696,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
           final DistributedRegionMXBean bean = service.getDistributedRegionMXBean(
               Region.SEPARATOR + REBALANCE_REGION_NAME);
           if (bean == null) {
-            LogWriterUtils.getLogWriter().info("Still probing for checkRegionMBeans ManagerMBean");
+            getLogWriter().info("Still probing for checkRegionMBeans ManagerMBean");
             return false;
           } else {
             // verify that bean is proper before executing tests
@@ -1704,32 +1713,34 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
           return "Probing for testRebalanceCommandForSimulateWithNoMember ManagerMBean";
         }
       };
-      Wait.waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+      waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
       DistributedRegionMXBean bean = ManagementService.getManagementService(getCache()).getDistributedRegionMXBean(
           "/" + REBALANCE_REGION_NAME);
       assertNotNull(bean);
     }
   };
 
+  @Test
   public void testRebalanceCommandForTimeOut() {
     setupTestRebalanceForEntireDS();
 
     //check if DistributedRegionMXBean is available so that command will not fail
     final VM manager = Host.getHost(0).getVM(0);
     manager.invoke(checkRegionMBeans);
-    LogWriterUtils.getLogWriter().info("testRebalanceCommandForTimeOut verified Mbean and executin command");
+    getLogWriter().info("testRebalanceCommandForTimeOut verified Mbean and executin command");
     String command = "rebalance --time-out=1";
     CommandResult cmdResult = executeCommand(command);
-    LogWriterUtils.getLogWriter().info("testRebalanceCommandForTimeOut just after executing " + cmdResult);
+    getLogWriter().info("testRebalanceCommandForTimeOut just after executing " + cmdResult);
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testRebalanceCommandForTimeOut stringResult : " + stringResult);
+      getLogWriter().info("testRebalanceCommandForTimeOut stringResult : " + stringResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testRebalanceCommandForTimeOut failed as did not get CommandResult");
     }
   }
 
+  @Test
   public void testRebalanceCommandForTimeOutForRegion() {
     setupTestRebalanceForEntireDS();
 
@@ -1737,22 +1748,23 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     final VM manager = Host.getHost(0).getVM(0);
     manager.invoke(checkRegionMBeans);
 
-    LogWriterUtils.getLogWriter().info("testRebalanceCommandForTimeOutForRegion verified Mbean and executin command");
+    getLogWriter().info("testRebalanceCommandForTimeOutForRegion verified Mbean and executin command");
 
     String command = "rebalance --time-out=1 --include-region=" + "/" + REBALANCE_REGION_NAME;
     CommandResult cmdResult = executeCommand(command);
 
-    LogWriterUtils.getLogWriter().info("testRebalanceCommandForTimeOutForRegion just after executing " + cmdResult);
+    getLogWriter().info("testRebalanceCommandForTimeOutForRegion just after executing " + cmdResult);
 
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testRebalanceCommandForTimeOutForRegion stringResult : " + stringResult);
+      getLogWriter().info("testRebalanceCommandForTimeOutForRegion stringResult : " + stringResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testRebalanceCommandForTimeOut failed as did not get CommandResult");
     }
   }
 
+  @Test
   public void testRebalanceCommandForSimulate() {
     setupTestRebalanceForEntireDS();
 
@@ -1760,19 +1772,20 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     final VM manager = Host.getHost(0).getVM(0);
     manager.invoke(checkRegionMBeans);
 
-    LogWriterUtils.getLogWriter().info("testRebalanceCommandForSimulate verified Mbean and executin command");
+    getLogWriter().info("testRebalanceCommandForSimulate verified Mbean and executin command");
     String command = "rebalance --simulate=true --include-region=" + "/" + REBALANCE_REGION_NAME;
     CommandResult cmdResult = executeCommand(command);
-    LogWriterUtils.getLogWriter().info("testRebalanceCommandForSimulate just after executing " + cmdResult);
+    getLogWriter().info("testRebalanceCommandForSimulate just after executing " + cmdResult);
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testRebalanceCommandForSimulate stringResult : " + stringResult);
+      getLogWriter().info("testRebalanceCommandForSimulate stringResult : " + stringResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testRebalanceCommandForSimulate failed as did not get CommandResult");
     }
   }
 
+  @Test
   public void testRebalanceCommandForSimulateWithNoMember() {
     setupTestRebalanceForEntireDS();
 
@@ -1780,22 +1793,23 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     final VM manager = Host.getHost(0).getVM(0);
     manager.invoke(checkRegionMBeans);
 
-    LogWriterUtils.getLogWriter().info("testRebalanceCommandForSimulateWithNoMember verified Mbean and executin command");
+    getLogWriter().info("testRebalanceCommandForSimulateWithNoMember verified Mbean and executin command");
 
     String command = "rebalance --simulate=true";
     CommandResult cmdResult = executeCommand(command);
 
-    LogWriterUtils.getLogWriter().info("testRebalanceCommandForSimulateWithNoMember just after executing " + cmdResult);
+    getLogWriter().info("testRebalanceCommandForSimulateWithNoMember just after executing " + cmdResult);
 
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testRebalanceCommandForSimulateWithNoMember stringResult : " + stringResult);
+      getLogWriter().info("testRebalanceCommandForSimulateWithNoMember stringResult : " + stringResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testRebalanceCommandForSimulateWithNoMember failed as did not get CommandResult");
     }
   }
 
+  @Test
   public void testRebalanceForIncludeRegionFunction() {
     // setup();
     setupWith2Regions();
@@ -1803,54 +1817,56 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     //check if DistributedRegionMXBean is available so that command will not fail
     final VM manager = Host.getHost(0).getVM(0);
     manager.invoke(checkRegionMBeans);
-    LogWriterUtils.getLogWriter().info("testRebalanceForIncludeRegionFunction verified Mbean and executin command");
+    getLogWriter().info("testRebalanceForIncludeRegionFunction verified Mbean and executin command");
     String command = "rebalance --include-region=" + "/" + REBALANCE_REGION_NAME + ",/" + REBALANCE_REGION2_NAME;
     CommandResult cmdResult = executeCommand(command);
-    LogWriterUtils.getLogWriter().info("testRebalanceForIncludeRegionFunction just after executing " + cmdResult);
+    getLogWriter().info("testRebalanceForIncludeRegionFunction just after executing " + cmdResult);
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testRebalanceForIncludeRegionFunction stringResult : " + stringResult);
+      getLogWriter().info("testRebalanceForIncludeRegionFunction stringResult : " + stringResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
     }
   }
 
+  @Test
   public void testSimulateForEntireDS() {
     setupTestRebalanceForEntireDS();
     //check if DistributedRegionMXBean is available so that command will not fail
     final VM manager = Host.getHost(0).getVM(0);
     manager.invoke(checkRegionMBeans);
 
-    LogWriterUtils.getLogWriter().info("testSimulateForEntireDS verified Mbean and executin command");
+    getLogWriter().info("testSimulateForEntireDS verified Mbean and executin command");
 
     String command = "rebalance --simulate=true";
 
     CommandResult cmdResult = executeCommand(command);
 
-    LogWriterUtils.getLogWriter().info("testSimulateForEntireDS just after executing " + cmdResult);
+    getLogWriter().info("testSimulateForEntireDS just after executing " + cmdResult);
 
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testSimulateForEntireDS stringResult : " + stringResult);
+      getLogWriter().info("testSimulateForEntireDS stringResult : " + stringResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
     }
   }
 
+  @Test
   public void testRebalanceForEntireDS() {
     setupTestRebalanceForEntireDS();
     //check if DistributedRegionMXBean is available so that command will not fail
     final VM manager = Host.getHost(0).getVM(0);
     manager.invoke(checkRegionMBeans);
-    LogWriterUtils.getLogWriter().info("testRebalanceForEntireDS verified Mbean and executin command");
+    getLogWriter().info("testRebalanceForEntireDS verified Mbean and executin command");
     String command = "rebalance";
     CommandResult cmdResult = executeCommand(command);
-    LogWriterUtils.getLogWriter().info("testRebalanceForEntireDS just after executing " + cmdResult);
+    getLogWriter().info("testRebalanceForEntireDS just after executing " + cmdResult);
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testRebalanceForEntireDS stringResult : " + stringResult);
+      getLogWriter().info("testRebalanceForEntireDS stringResult : " + stringResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
@@ -1901,14 +1917,14 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
   private static void printCommandOutput(CommandResult cmdResult) {
     assertNotNull(cmdResult);
-    LogWriterUtils.getLogWriter().info("Command Output : ");
+    getLogWriter().info("Command Output : ");
     StringBuilder sb = new StringBuilder();
     cmdResult.resetToFirstLine();
     while (cmdResult.hasNextLine()) {
       sb.append(cmdResult.nextLine()).append(DataCommandRequest.NEW_LINE);
     }
-    LogWriterUtils.getLogWriter().info(sb.toString());
-    LogWriterUtils.getLogWriter().info("");
+    getLogWriter().info(sb.toString());
+    getLogWriter().info("");
   }
 
   public static class Value1WithValue2 extends Value1 {
@@ -1928,6 +1944,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testRebalanceForExcludeRegionFunction() {
     setupWith2Regions();
 
@@ -1935,15 +1952,15 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     final VM manager = Host.getHost(0).getVM(0);
     manager.invoke(checkRegionMBeans);
 
-    LogWriterUtils.getLogWriter().info("testRebalanceForExcludeRegionFunction verified Mbean and executing command");
+    getLogWriter().info("testRebalanceForExcludeRegionFunction verified Mbean and executing command");
 
     String command = "rebalance --exclude-region=" + "/" + REBALANCE_REGION2_NAME;
-    LogWriterUtils.getLogWriter().info("testRebalanceForExcludeRegionFunction command : " + command);
+    getLogWriter().info("testRebalanceForExcludeRegionFunction command : " + command);
     CommandResult cmdResult = executeCommand(command);
-    LogWriterUtils.getLogWriter().info("testRebalanceForExcludeRegionFunction just after executing " + cmdResult);
+    getLogWriter().info("testRebalanceForExcludeRegionFunction just after executing " + cmdResult);
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testRebalanceForExcludeRegionFunction stringResult : " + stringResult);
+      getLogWriter().info("testRebalanceForExcludeRegionFunction stringResult : " + stringResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
@@ -1966,19 +1983,19 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
             ManagerMXBean bean1 = service.getManagerMXBean();
             DistributedRegionMXBean bean2 = service.getDistributedRegionMXBean(regionName);
             if (bean1 == null) {
-              LogWriterUtils.getLogWriter().info("waitForListClientMbean Still probing for ManagerMBean");
+              getLogWriter().info("waitForListClientMbean Still probing for ManagerMBean");
               return false;
             } else {
-              LogWriterUtils.getLogWriter().info("waitForListClientMbean Still probing for DistributedRegionMXBean=" + bean2);
+              getLogWriter().info("waitForListClientMbean Still probing for DistributedRegionMXBean=" + bean2);
               if (bean2 == null) {
                 bean2 = service.getDistributedRegionMXBean(Region.SEPARATOR + regionName);
               }
               if (bean2 == null) {
-                LogWriterUtils.getLogWriter().info(
+                getLogWriter().info(
                     "waitForListClientMbean Still probing for DistributedRegionMXBean with separator = " + bean2);
                 return false;
               } else {
-                LogWriterUtils.getLogWriter().info(
+                getLogWriter().info(
                     "waitForListClientMbean Still probing for DistributedRegionMXBean with separator Not null  " + bean2.getMembers().length);
                 if (bean2.getMembers().length > 1) {
                   return true;
@@ -1995,7 +2012,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
           }
         };
 
-        Wait.waitForCriterion(waitForMaangerMBean, 30000, 2000, true);
+        waitForCriterion(waitForMaangerMBean, 30000, 2000, true);
         DistributedRegionMXBean bean = service.getDistributedRegionMXBean(regionName);
         if (bean == null) {
           bean = service.getDistributedRegionMXBean(Region.SEPARATOR + regionName);
@@ -2006,6 +2023,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
 
   }
 
+  @Test
   public void testRegionsViaMbeanAndFunctions() {
 
     setupForGetPutRemoveLocateEntry("tesSimplePut");
@@ -2031,7 +2049,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       }
     });
 
-    LogWriterUtils.getLogWriter().info("testRegionsViaMbeanAndFunctions memSizeFromMbean= " + memSizeFromMbean);
+    getLogWriter().info("testRegionsViaMbeanAndFunctions memSizeFromMbean= " + memSizeFromMbean);
 
     String memSizeFromFunctionCall = (String) manager.invoke(new SerializableCallable() {
       public Object call() {
@@ -2041,10 +2059,11 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       }
     });
 
-    LogWriterUtils.getLogWriter().info("testRegionsViaMbeanAndFunctions memSizeFromFunctionCall= " + memSizeFromFunctionCall);
+    getLogWriter().info("testRegionsViaMbeanAndFunctions memSizeFromFunctionCall= " + memSizeFromFunctionCall);
     assertTrue(memSizeFromFunctionCall.equals(memSizeFromMbean));
   }
 
+  @Test
   public void testRegionsViaMbeanAndFunctionsForPartRgn() {
     setupWith2Regions();
     waitForListClientMbean(REBALANCE_REGION_NAME);
@@ -2070,7 +2089,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       }
     });
 
-    LogWriterUtils.getLogWriter().info("testRegionsViaMbeanAndFunctionsForPartRgn memSizeFromMbean= " + memSizeFromMbean);
+    getLogWriter().info("testRegionsViaMbeanAndFunctionsForPartRgn memSizeFromMbean= " + memSizeFromMbean);
 
     String memSizeFromFunctionCall = (String) manager.invoke(new SerializableCallable() {
       public Object call() {
@@ -2079,7 +2098,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       }
     });
 
-    LogWriterUtils.getLogWriter().info(
+    getLogWriter().info(
         "testRegionsViaMbeanAndFunctionsForPartRgn memSizeFromFunctionCall= " + memSizeFromFunctionCall);
     assertTrue(memSizeFromFunctionCall.equals(memSizeFromMbean));
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java
index 6704979..19a354e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.java
@@ -16,11 +16,18 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
 import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheLoader;
 import com.gemstone.gemfire.cache.CacheLoaderException;
@@ -40,12 +47,11 @@ import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData;
 import com.gemstone.gemfire.management.internal.cli.result.ResultData;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 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.SerializableRunnableIF;
 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.DistributedTest;
 
 /**
  * The GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest class is test suite of test cases testing the Gfsh
@@ -57,6 +63,7 @@ import com.gemstone.gemfire.test.dunit.WaitCriterion;
  * @since 8.0
  */
 @SuppressWarnings("unused")
+@Category(DistributedTest.class)
 public class GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest extends CliCommandTestBase {
 
   private static final String GEMFIRE_MANAGER_NAME = "GemManagerNode";
@@ -82,10 +89,6 @@ public class GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest extends C
     return buffer.toString();
   }
 
-  public GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest(final String testName) {
-    super(testName);
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     Properties managerDistributedSystemProperties = createDistributedSystemProperties(GEMFIRE_MANAGER_NAME);
@@ -171,7 +174,7 @@ public class GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest extends C
           }
         };
 
-        Wait.waitForCriterion(waitOnManagerCriterion, 30000, 2000, true);
+        waitForCriterion(waitOnManagerCriterion, 30000, 2000, true);
       }
     });
   }
@@ -194,7 +197,7 @@ public class GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest extends C
 
   protected void log(final String tag, final String message) {
     //System.out.printf("%1$s (%2$s)%n", tag, message);
-    LogWriterUtils.getLogWriter().info(String.format("%1$s (%2$s)%n", tag, message));
+    getLogWriter().info(String.format("%1$s (%2$s)%n", tag, message));
   }
 
   protected CommandResult runCommand(final String command) {
@@ -218,6 +221,7 @@ public class GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest extends C
     }
   }
 
+  @Test
   public void testGetOnCacheMiss() {
     doHousekeeping();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java
index 49f405c..344d116 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/HTTPServiceSSLSupportJUnitTest.java
@@ -16,23 +16,23 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static org.junit.Assert.*;
+
 import java.io.File;
 import java.util.Properties;
 
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import static org.junit.Assert.*;
-
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
-import com.gemstone.gemfire.util.test.TestUtil;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.util.test.TestUtil;
 
 /**
- * 
  * @since 8.1
  */
 @Category(IntegrationTest.class)
@@ -68,7 +68,9 @@ public class HTTPServiceSSLSupportJUnitTest {
     return sb.toString();
   }
 
-  public void _testSSLWithClusterSSL() throws Exception {
+  @Ignore("disabled for unknown reason")
+  @Test
+  public void testSSLWithClusterSSL() throws Exception {
 
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
index 431c6b3..123c3bc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
@@ -16,6 +16,15 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.DiskStoreFactory;
@@ -43,11 +52,10 @@ 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.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Properties;
-
+@Category({ DistributedTest.class, FlakyTest.class }) // see GEODE-689, GEODE-1048
 public class IndexCommandsDUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
@@ -57,11 +65,6 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
   private static final String parRegPersName = "ParRegPers";
   private static final String repRegPersName = "RepRegPer";
 
-  public IndexCommandsDUnitTest(String name) {
-    super(name);
-    // TODO Auto-generated constructor stub
-  }
-
   Region<?, ?> createParReg(String regionName, Cache cache, Class keyConstraint, Class valueConstraint) {
     RegionFactory regionFactory = cache.createRegionFactory();
     regionFactory.setDataPolicy(DataPolicy.PARTITION);
@@ -130,6 +133,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     return regionFactory.create(regionName);
   }
 
+  @Test
   public void testCreateKeyIndexOnRegionWithPersistence() {
     setupSystemPersist();
 
@@ -159,6 +163,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     assertTrue(Status.OK.equals(commandResult.getStatus()));
   }
 
+  @Test
   public void testCreateAndDestroyIndex() {
     setupSystem();
     /***
@@ -200,6 +205,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     assertFalse(resultAsString.contains(indexName));
   }
 
+  @Test
   public void testCreateIndexMultipleIterators() {
     setupSystem();
 
@@ -224,6 +230,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     assertTrue(resultAsString.contains(indexName));
   }
 
+  @Test
   public void testCreateMultipleIndexes() {
     setupSystem();
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEFINE_INDEX);
@@ -257,6 +264,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     assertTrue(resultAsString.contains(indexName));
   }
 
+  @Test
   public void testClearMultipleIndexes() {
     setupSystem();
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DEFINE_INDEX);
@@ -290,6 +298,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     assertTrue(!resultAsString.contains(indexName));
   }
 
+  @Test
   public void testCreateAndDestroyIndexOnMember() {
     setupSystem();
     /***
@@ -339,6 +348,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     assertFalse(resultAsString.contains(VM1Name));
   }
 
+  @Test
   public void testCreateAndDestroyIndexOnGroup() {
     setupSystem();
     /***
@@ -406,6 +416,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testCreateAndDestroyIndexWithIncorrectInput() {
     setupSystem();
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
@@ -524,6 +535,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     assertTrue(commandResult.getStatus().equals(Status.ERROR));
   }
 
+  @Test
   public void testDestroyIndexWithoutIndexName() {
     setupSystem();
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_INDEX);
@@ -585,6 +597,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
   /**
    * Asserts that creating and destroying indexes correctly updates the shared configuration.
    */
+  @Test
   public void testCreateDestroyUpdatesSharedConfig() {
     disconnectAllFromDS();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsJUnitTest.java
index bf77aba..ce36401 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsJUnitTest.java
@@ -24,6 +24,15 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
+import org.jmock.Expectations;
+import org.jmock.Mockery;
+import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.execute.Execution;
 import com.gemstone.gemfire.cache.execute.FunctionInvocationTargetException;
@@ -35,15 +44,6 @@ import com.gemstone.gemfire.management.internal.cli.domain.IndexDetails;
 import com.gemstone.gemfire.management.internal.cli.functions.ListIndexFunction;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.jmock.Expectations;
-import org.jmock.Mockery;
-import org.jmock.lib.legacy.ClassImposteriser;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 /**
  * The IndexCommandsJUnitTest class is a test suite of test cases testing the contract and functionality of the
  * IndexCommands class.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java
index 744824f..6466033 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeDiskStoreCommandsDUnitTest.java
@@ -16,9 +16,15 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
 import java.io.Serializable;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.DiskStore;
@@ -29,10 +35,10 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 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.SerializableRunnableIF;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * The ListAndDescribeDiskStoreCommandsDUnitTest class is a test suite of functional tests cases testing the proper
@@ -42,6 +48,7 @@ import com.gemstone.gemfire.test.dunit.VM;
  * @see com.gemstone.gemfire.management.internal.cli.commands.DiskStoreCommands
  * @since 7.0
  */
+@Category(DistributedTest.class)
 public class ListAndDescribeDiskStoreCommandsDUnitTest extends CliCommandTestBase {
 
   protected static String toString(final Result result) {
@@ -57,10 +64,6 @@ public class ListAndDescribeDiskStoreCommandsDUnitTest extends CliCommandTestBas
     return buffer.toString().trim();
   }
 
-  public ListAndDescribeDiskStoreCommandsDUnitTest(final String testName) {
-    super(testName);
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     createDefaultSetup(null);
@@ -89,7 +92,7 @@ public class ListAndDescribeDiskStoreCommandsDUnitTest extends CliCommandTestBas
   protected Properties createDistributedSystemProperties(final String gemfireName) {
     final Properties distributedSystemProperties = new Properties();
 
-    distributedSystemProperties.setProperty(DistributionConfig.LOG_LEVEL_NAME, LogWriterUtils.getDUnitLogLevel());
+    distributedSystemProperties.setProperty(DistributionConfig.LOG_LEVEL_NAME, getDUnitLogLevel());
     distributedSystemProperties.setProperty(DistributionConfig.NAME_NAME, gemfireName);
 
     return distributedSystemProperties;
@@ -121,23 +124,26 @@ public class ListAndDescribeDiskStoreCommandsDUnitTest extends CliCommandTestBas
     });
   }
 
+  @Test
   public void testListDiskStore() throws Exception {
     final Result result = executeCommand(CliStrings.LIST_DISK_STORE);
 
     assertNotNull(result);
-    LogWriterUtils.getLogWriter().info(toString(result));
+    getLogWriter().info(toString(result));
     assertEquals(Result.Status.OK, result.getStatus());
   }
 
+  @Test
   public void testDescribeDiskStore() throws Exception {
     final Result result = executeCommand(
         CliStrings.DESCRIBE_DISK_STORE + " --member=producerServer --name=producerData");
 
     assertNotNull(result);
-    LogWriterUtils.getLogWriter().info(toString(result));
+    getLogWriter().info(toString(result));
     assertEquals(Result.Status.OK, result.getStatus());
   }
 
+  @Test
   public void testDescribeDiskStoreWithInvalidMemberName() throws Exception {
     final Result commandResult = executeCommand(
         CliStrings.DESCRIBE_DISK_STORE + " --member=badMemberName --name=producerData");
@@ -148,6 +154,7 @@ public class ListAndDescribeDiskStoreCommandsDUnitTest extends CliCommandTestBas
         toString(commandResult));
   }
 
+  @Test
   public void testDescribeDiskStoreWithInvalidDiskStoreName() {
     final Result commandResult = executeCommand(
         CliStrings.DESCRIBE_DISK_STORE + " --member=producerServer --name=badDiskStoreName");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java
index 2d11491..f495944 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListAndDescribeRegionDUnitTest.java
@@ -16,6 +16,14 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
+import java.util.Properties;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.EvictionAction;
 import com.gemstone.gemfire.cache.EvictionAttributes;
@@ -35,19 +43,13 @@ import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 import com.gemstone.gemfire.management.internal.cli.util.RegionAttributesNames;
 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.junit.categories.DistributedTest;
 
-import java.util.Properties;
-
+@Category(DistributedTest.class)
 public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
 
-  public ListAndDescribeRegionDUnitTest(String name) {
-    super(name);
-    // TODO Auto-generated constructor stub
-  }
-
   private static final String REGION1 = "region1";
   private static final String REGION2 = "region2";
   private static final String REGION3 = "region3";
@@ -186,15 +188,15 @@ public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
     dataRegionFactory.create(REGION3);
   }
 
-
+  @Test
   public void testListRegion() {
     setupSystem();
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.LIST_REGION);
     String commandString = csb.toString();
     CommandResult commandResult = executeCommand(commandString);
     String commandResultAsString = commandResultToString(commandResult);
-    LogWriterUtils.getLogWriter().info("Command String : " + commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultAsString);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
     assertEquals(Status.OK, commandResult.getStatus());
     assertTrue(commandResultAsString.contains(PR1));
     assertTrue(commandResultAsString.contains(LOCALREGIONONMANAGER));
@@ -208,8 +210,8 @@ public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
     commandString = csb.toString();
     commandResult = executeCommand(commandString);
     commandResultAsString = commandResultToString(commandResult);
-    LogWriterUtils.getLogWriter().info("Command String : " + commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultAsString);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
     assertEquals(Status.OK, commandResult.getStatus());
     assertTrue(commandResultAsString.contains(PR1));
     assertTrue(commandResultAsString.contains(LOCALREGIONONMANAGER));
@@ -219,8 +221,8 @@ public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
     commandString = csb.toString();
     commandResult = executeCommand(commandString);
     commandResultAsString = commandResultToString(commandResult);
-    LogWriterUtils.getLogWriter().info("Command String : " + commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultAsString);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
     assertEquals(Status.OK, commandResult.getStatus());
     assertTrue(commandResultAsString.contains(PR1));
     assertTrue(commandResultAsString.contains(REGION1));
@@ -233,8 +235,8 @@ public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
     commandString = csb.toString();
     commandResult = executeCommand(commandString);
     commandResultAsString = commandResultToString(commandResult);
-    LogWriterUtils.getLogWriter().info("Command String : " + commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultAsString);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
     assertEquals(Status.OK, commandResult.getStatus());
     assertTrue(commandResultAsString.contains(PR1));
     assertTrue(commandResultAsString.contains(LOCALREGIONONMANAGER));
@@ -244,8 +246,8 @@ public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
     commandString = csb.toString();
     commandResult = executeCommand(commandString);
     commandResultAsString = commandResultToString(commandResult);
-    LogWriterUtils.getLogWriter().info("Command String : " + commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultAsString);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
     assertEquals(Status.OK, commandResult.getStatus());
     assertTrue(commandResultAsString.contains(PR1));
     assertTrue(commandResultAsString.contains(REGION1));
@@ -254,6 +256,7 @@ public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
     assertTrue(commandResultAsString.contains(SUBREGION1A));
   }
 
+  @Test
   public void testDescribeRegion() {
     setupSystem();
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.DESCRIBE_REGION);
@@ -261,8 +264,8 @@ public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
     String commandString = csb.toString();
     CommandResult commandResult = executeCommand(commandString);
     String commandResultAsString = commandResultToString(commandResult);
-    LogWriterUtils.getLogWriter().info("Command String : " + commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultAsString);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
     assertEquals(Status.OK, commandResult.getStatus());
     assertTrue(commandResultAsString.contains(PR1));
     assertTrue(commandResultAsString.contains("Server1"));
@@ -272,8 +275,8 @@ public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
     commandString = csb.toString();
     commandResult = executeCommand(commandString);
     commandResultAsString = commandResultToString(commandResult);
-    LogWriterUtils.getLogWriter().info("Command String : " + commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultAsString);
+    getLogWriter().info("Command String : " + commandString);
+    getLogWriter().info("Output : \n" + commandResultAsString);
     assertEquals(Status.OK, commandResult.getStatus());
     assertTrue(commandResultAsString.contains(LOCALREGIONONMANAGER));
     assertTrue(commandResultAsString.contains("Manager"));
@@ -283,6 +286,7 @@ public class ListAndDescribeRegionDUnitTest extends CliCommandTestBase {
    * Asserts that a describe region command issued on a region with compression returns the correct non default region
    * attribute for compression and the correct codec value.
    */
+  @Test
   public void testDescribeRegionWithCompressionCodec() {
     final String regionName = "compressedRegion";
     VM vm = Host.getHost(0).getVM(1);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java
index caa33fb..deb8860 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ListIndexCommandDUnitTest.java
@@ -16,6 +16,9 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -28,6 +31,9 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.Region;
@@ -45,10 +51,10 @@ import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.domain.IndexDetails;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 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.SerializableRunnableIF;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * The ListIndexCommandDUnitTest class is distributed test suite of test cases for testing the index-based GemFire shell
@@ -59,6 +65,7 @@ import com.gemstone.gemfire.test.dunit.VM;
  * @since 7.0
  */
 @SuppressWarnings("unused")
+@Category(DistributedTest.class)
 public class ListIndexCommandDUnitTest extends CliCommandTestBase {
 
   protected static final int DEFAULT_REGION_INITIAL_CAPACITY = 10000;
@@ -78,11 +85,6 @@ public class ListIndexCommandDUnitTest extends CliCommandTestBase {
     return buffer.toString();
   }
 
-
-  public ListIndexCommandDUnitTest(final String testName) {
-    super(testName);
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     createDefaultSetup(null);
@@ -136,7 +138,7 @@ public class ListIndexCommandDUnitTest extends CliCommandTestBase {
   protected Properties createDistributedSystemProperties(final String gemfireName) {
     final Properties distributedSystemProperties = new Properties();
 
-    distributedSystemProperties.setProperty(DistributionConfig.LOG_LEVEL_NAME, LogWriterUtils.getDUnitLogLevel());
+    distributedSystemProperties.setProperty(DistributionConfig.LOG_LEVEL_NAME, getDUnitLogLevel());
     distributedSystemProperties.setProperty(DistributionConfig.NAME_NAME, gemfireName);
 
     return distributedSystemProperties;
@@ -174,7 +176,7 @@ public class ListIndexCommandDUnitTest extends CliCommandTestBase {
               }
             }
           } catch (Exception e) {
-            LogWriterUtils.getLogWriter().error(
+            getLogWriter().error(
                 String.format("Error occurred creating Index (%1$s) on Region (%2$s) - (%3$s)", indexName,
                     region.getFullPath(), e.getMessage()));
           }
@@ -276,11 +278,11 @@ public class ListIndexCommandDUnitTest extends CliCommandTestBase {
   @SuppressWarnings("unchecked")
   protected <T extends Comparable<T>, B extends AbstractBean<T>> B query(final Cache cache, final String queryString) {
     try {
-      LogWriterUtils.getLogWriter().info(String.format("Running Query (%1$s) in GemFire...", queryString));
+      getLogWriter().info(String.format("Running Query (%1$s) in GemFire...", queryString));
 
       final SelectResults<B> results = (SelectResults<B>) cache.getQueryService().newQuery(queryString).execute();
 
-      LogWriterUtils.getLogWriter().info(
+      getLogWriter().info(
           String.format("Running Query (%1$s) in GemFire returned (%2$d) result(s).", queryString, results.size()));
 
       return (results.iterator().hasNext() ? results.iterator().next() : null);
@@ -292,12 +294,12 @@ public class ListIndexCommandDUnitTest extends CliCommandTestBase {
   protected <T extends Comparable<T>, B extends AbstractBean<T>> B query(final Region<T, B> region,
       final String queryPredicate) {
     try {
-      LogWriterUtils.getLogWriter().info(
+      getLogWriter().info(
           String.format("Running Query (%1$s) on Region (%2$s)...", queryPredicate, region.getFullPath()));
 
       final SelectResults<B> results = region.query(queryPredicate);
 
-      LogWriterUtils.getLogWriter().info(
+      getLogWriter().info(
           String.format("Running Query (%1$s) on Region (%2$s) returned (%3$d) result(s).", queryPredicate,
               region.getFullPath(), results.size()));
 
@@ -309,11 +311,12 @@ public class ListIndexCommandDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testListIndex() throws Exception {
     final Result result = executeCommand(CliStrings.LIST_INDEX + " --" + CliStrings.LIST_INDEX__STATS);
 
     assertNotNull(result);
-    LogWriterUtils.getLogWriter().info(toString(result));
+    getLogWriter().info(toString(result));
     assertEquals(Result.Status.OK, result.getStatus());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
index 44cead8..d2bdcba 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
@@ -16,6 +16,21 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.NetworkUtils.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.EvictionAction;
@@ -26,7 +41,6 @@ import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -39,24 +53,16 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.remote.CommandProcessor;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 import com.gemstone.gemfire.test.dunit.Host;
-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.dunit.cache.internal.JUnit4CacheTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-/****
- */
+@Category(DistributedTest.class)
+public class MemberCommandsDUnitTest extends JUnit4CacheTestCase {
 
-public class MemberCommandsDUnitTest extends CacheTestCase {
   private static final long serialVersionUID = 1L;
+
   private static final Map<String, String> EMPTY_ENV = Collections.emptyMap();
   private static final String REGION1 = "region1";
   private static final String REGION2 = "region2";
@@ -67,10 +73,6 @@ public class MemberCommandsDUnitTest extends CacheTestCase {
   private static final String PR1 = "PartitionedRegion1";
   private static final String PR2 = "ParitionedRegion2";
 
-  public MemberCommandsDUnitTest(String name) {
-    super(name);
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     // This test does not require an actual Gfsh connection to work, however when run as part of a suite, prior tests
@@ -179,7 +181,7 @@ public class MemberCommandsDUnitTest extends CacheTestCase {
     Properties props = new Properties();
 
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, NetworkUtils.getServerHostName(host) + "[" + locatorPort + "]");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, getServerHostName(host) + "[" + locatorPort + "]");
     props.setProperty(DistributionConfig.LOG_LEVEL_NAME, "info");
     props.setProperty(DistributionConfig.STATISTIC_SAMPLING_ENABLED_NAME, "true");
     props.setProperty(DistributionConfig.ENABLE_TIME_STATISTICS_NAME, "true");
@@ -196,26 +198,28 @@ public class MemberCommandsDUnitTest extends CacheTestCase {
     final Cache cache = getCache();
   }
 
-  /***
+  /**
    * Tests the execution of "list member" command which should list out all the members in the DS
    *
    * @throws IOException
    * @throws ClassNotFoundException
    */
+  @Test
   public void testListMemberAll() throws IOException, ClassNotFoundException {
     setupSystem();
     CommandProcessor commandProcessor = new CommandProcessor();
     Result result = commandProcessor.createCommandStatement(CliStrings.LIST_MEMBER, EMPTY_ENV).process();
-    LogWriterUtils.getLogWriter().info("#SB" + getResultAsString(result));
+    getLogWriter().info("#SB" + getResultAsString(result));
     assertEquals(true, result.getStatus().equals(Status.OK));
   }
 
-  /****
+  /**
    * Tests the execution of "list member" command, when no cache is created
    *
    * @throws IOException
    * @throws ClassNotFoundException
    */
+  @Test
   public void testListMemberWithNoCache() throws IOException, ClassNotFoundException {
     final Host host = Host.getHost(0);
     final VM[] servers = {host.getVM(0), host.getVM(1)};
@@ -229,35 +233,37 @@ public class MemberCommandsDUnitTest extends CacheTestCase {
       CommandProcessor commandProcessor = new CommandProcessor();
       Result result = commandProcessor.createCommandStatement(CliStrings.LIST_MEMBER, EMPTY_ENV).process();
 
-      LogWriterUtils.getLogWriter().info("#SB" + getResultAsString(result));
+      getLogWriter().info("#SB" + getResultAsString(result));
       assertEquals(true, result.getStatus().equals(Status.ERROR));
     } finally {
       locator.stop(); // fix for bug 46562
     }
   }
 
-  /***
+  /**
    * Tests list member --group=G1
    *
    * @throws IOException
    * @throws ClassNotFoundException
    */
+  @Test
   public void testListMemberWithGroups() throws IOException, ClassNotFoundException {
     setupSystem();
     CommandProcessor commandProcessor = new CommandProcessor();
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.LIST_MEMBER);
     csb.addOption(CliStrings.LIST_MEMBER__GROUP, "G1");
     Result result = commandProcessor.createCommandStatement(csb.toString(), EMPTY_ENV).process();
-    LogWriterUtils.getLogWriter().info("#SB" + getResultAsString(result));
+    getLogWriter().info("#SB" + getResultAsString(result));
     assertEquals(true, result.getStatus().equals(Status.OK));
   }
 
-  /***
+  /**
    * Tests the "describe member" command for all the members in the DS
    *
    * @throws IOException
    * @throws ClassNotFoundException
    */
+  @Test
   public void testDescribeMember() throws IOException, ClassNotFoundException {
     setupSystem();
     CommandProcessor commandProcessor = new CommandProcessor();
@@ -271,7 +277,7 @@ public class MemberCommandsDUnitTest extends CacheTestCase {
       Result result = commandProcessor.createCommandStatement("describe member --name=" + member.getId(),
           EMPTY_ENV).process();
       assertEquals(true, result.getStatus().equals(Status.OK));
-      LogWriterUtils.getLogWriter().info("#SB" + getResultAsString(result));
+      getLogWriter().info("#SB" + getResultAsString(result));
       //assertEquals(true, result.getStatus().equals(Status.OK));
     }
   }


[14/32] incubator-geode git commit: GEODE-1018: Retrieve queues only once gateway listener to avoid a race

Posted by ji...@apache.org.
GEODE-1018: Retrieve queues only once gateway listener to avoid a race

The second call to getQueues could have returned null due to a
concurrent change. By fetching the queues only once we avoid a race.


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

Branch: refs/heads/feature/GEODE-17-2
Commit: a885ac140b50d4f3ec74ede9fcdff98623d8dc81
Parents: bbf705e
Author: Dan Smith <up...@apache.org>
Authored: Wed Apr 13 10:12:29 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Wed Apr 13 16:54:58 2016 -0700

----------------------------------------------------------------------
 .../serial/SerialSecondaryGatewayListener.java   | 19 ++++++++-----------
 1 file changed, 8 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a885ac14/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialSecondaryGatewayListener.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialSecondaryGatewayListener.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialSecondaryGatewayListener.java
index 5cb0ec0..8250270 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialSecondaryGatewayListener.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialSecondaryGatewayListener.java
@@ -16,10 +16,13 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.serial;
 
+import java.util.Set;
+
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.cache.EntryEvent;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
+import com.gemstone.gemfire.internal.cache.RegionQueue;
 import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySender;
 import com.gemstone.gemfire.internal.cache.wan.GatewaySenderEventImpl;
 import com.gemstone.gemfire.internal.logging.LogService;
@@ -55,11 +58,8 @@ public class SerialSecondaryGatewayListener extends CacheListenerAdapter
    }
    // There is a small window where queue has not been created fully yet. 
    // The underlying region of the queue is created, and it receives afterDestroy callback
-   if (this.sender.getQueues() != null && !this.sender.getQueues().isEmpty()) {
-//     int size = 0;
-//     for(RegionQueue q: this.sender.getQueues()) {
-//       size += q.size();
-//     }
+   final Set<RegionQueue> queues = this.sender.getQueues();
+   if (queues != null && !queues.isEmpty()) {
      this.sender.getStatistics().incQueueSize();
    }
    // fix bug 35730
@@ -76,12 +76,9 @@ public class SerialSecondaryGatewayListener extends CacheListenerAdapter
    }
     // fix bug 37603
     // There is a small window where queue has not been created fully yet. The region is created, and it receives afterDestroy callback.
-   
-   if (this.sender.getQueues() != null && !this.sender.getQueues().isEmpty()) {
-//     int size = 0;
-//     for(RegionQueue q: this.sender.getQueues()) {
-//       size += q.size();
-//     }
+
+   final Set<RegionQueue> queues = this.sender.getQueues();
+   if (queues != null && !queues.isEmpty()) {
      this.sender.getStatistics().decQueueSize();
    }
 


[05/32] incubator-geode git commit: GEODE-1218: Marking jna as not optional

Posted by ji...@apache.org.
GEODE-1218: Marking jna as not optional

JNA is used for common features like offheap memory and disk stores, so
it should not be marked optional.


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

Branch: refs/heads/feature/GEODE-17-2
Commit: fbee35cc424f80d5f9ed316da99e0caeb5600ddc
Parents: 69cd4a7
Author: Dan Smith <up...@apache.org>
Authored: Tue Apr 12 13:43:14 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Wed Apr 13 10:08:42 2016 -0700

----------------------------------------------------------------------
 geode-core/build.gradle | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fbee35cc/geode-core/build.gradle
----------------------------------------------------------------------
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index 8216d2a..2206018 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -61,9 +61,7 @@ dependencies {
   compile ('mx4j:mx4j-tools:' + project.'mx4j.version') { 
     ext.optional = true;
   }
-  compile ('net.java.dev.jna:jna:' + project.'jna.version') {
-    ext.optional = true
-  }
+  compile ('net.java.dev.jna:jna:' + project.'jna.version')
   provided ('org.apache.hadoop:hadoop-common:' + project.'hadoop.version') {
     transitive=false
   }


[20/32] incubator-geode git commit: GEODE-1162: convert all CLI command DUnit tests to JUnit 4

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java
index 2c93ddf..d016843 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit4DistributedTestCase.java
@@ -26,6 +26,12 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+
 import com.gemstone.gemfire.admin.internal.AdminDistributedSystemImpl;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
@@ -61,18 +67,11 @@ import com.gemstone.gemfire.test.dunit.Invoke;
 import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 import com.gemstone.gemfire.test.junit.rules.serializable.SerializableTestName;
-import org.apache.logging.log4j.Logger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
 
 /**
  * This class is the base class for all distributed tests using JUnit 4.
- *
- * TODO: make this class abstract when JUnit3DistributedTestCase is deleted
  */
-public class JUnit4DistributedTestCase implements DistributedTestFixture, Serializable {
+public abstract class JUnit4DistributedTestCase implements DistributedTestFixture, Serializable {
 
   private static final Logger logger = LogService.getLogger();
 
@@ -98,7 +97,7 @@ public class JUnit4DistributedTestCase implements DistributedTestFixture, Serial
    * no-arg constructor.
    */
   public JUnit4DistributedTestCase() {
-    this((DistributedTestFixture)null);
+    this(null);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3BasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3BasicDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3BasicDUnitTest.java
deleted file mode 100644
index 5130b5c..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3BasicDUnitTest.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.test.dunit.internal.tests;
-
-import static com.gemstone.gemfire.test.dunit.Invoke.*;
-
-import java.util.Properties;
-
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.DUnitEnv;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.RMIException;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.internal.JUnit3DistributedTestCase;
-
-/**
- * This class tests the basic functionality of the distributed unit
- * test framework.
- */
-public class JUnit3BasicDUnitTest extends JUnit3DistributedTestCase {
-
-  private static Properties bindings;
-
-  public JUnit3BasicDUnitTest(String name) {
-    super(name);
-  }
-
-  @Override
-  public void postSetUp() throws Exception {
-    bindings = new Properties();
-    invokeInEveryVM(() -> bindings = new Properties());
-  }
-
-  @Override
-  public void postTearDown() throws Exception {
-    bindings = null;
-    invokeInEveryVM(() -> bindings = null);
-  }
-
-  public void testPreconditions() {
-    invokeInEveryVM(() -> assertNotNull("getUniqueName() must not return null", getUniqueName()));
-    invokeInEveryVM(() -> assertNotNull("bindings must not be null", bindings));
-  }
-
-  /**
-   * Tests how the Hydra framework handles an error
-   */
-  public void ignore_testDontCatchRemoteException() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    vm.invoke(() -> remoteThrowException());
-  }
-
-  public void testRemoteInvocationWithException() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(() -> remoteThrowException());
-      fail("Should have thrown a BasicTestException");
-
-    } catch (RMIException ex) {
-      assertTrue(ex.getCause() instanceof BasicTestException);
-    }
-  }
-
-  public void testInvokeWithLambda() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    int vm0Num = vm0.invoke(() -> DUnitEnv.get().getVMID());
-    int vm1Num = vm1.invoke(() -> DUnitEnv.get().getVMID());
-
-    assertEquals(0, vm0Num);
-    assertEquals(1, vm1Num);
-  }
-
-  public void testInvokeLambdaAsync() throws Throwable {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-
-    AsyncInvocation<Integer> async0 = vm0.invokeAsync(() -> DUnitEnv.get().getVMID());
-    int vm0num = async0.getResult();
-
-    assertEquals(0, vm0num);
-  }
-
-  public void testInvokeWithNamedLambda() {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    int vm0Num = vm0.invoke("getVMID", () -> DUnitEnv.get().getVMID());
-    int vm1Num = vm1.invoke("getVMID", () -> DUnitEnv.get().getVMID());
-
-    assertEquals(0, vm0Num);
-    assertEquals(1, vm1Num);
-  }
-
-  public void testInvokeNamedLambdaAsync() throws Throwable {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-
-    AsyncInvocation<Integer> async0 = vm0.invokeAsync("getVMID", () -> DUnitEnv.get().getVMID());
-    int vm0num = async0.getResult();
-
-    assertEquals(0, vm0num);
-  }
-
-  // Test was never implemented
-  public void ignore_testRemoteInvocationBoolean() {
-  }
-
-  public void testRemoteInvokeAsync() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    String name = getUniqueName();
-    String value = "Hello";
-
-    AsyncInvocation ai = vm.invokeAsync(() -> remoteBind(name, value));
-    ai.join();
-    // TODO shouldn't we call fail() here?
-    if (ai.exceptionOccurred()) {
-      Assert.fail("remoteBind failed", ai.getException());
-    }
-
-    ai = vm.invokeAsync(() -> remoteValidateBind(name, value ));
-    ai.join();
-    if (ai.exceptionOccurred()) {
-      Assert.fail("remoteValidateBind failed", ai.getException());
-    }
-  }
-
-  public void testRemoteInvokeAsyncWithException() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-
-    AsyncInvocation ai = vm.invokeAsync(() -> remoteThrowException());
-    ai.join();
-    assertTrue(ai.exceptionOccurred());
-    Throwable ex = ai.getException();
-    assertTrue(ex instanceof BasicTestException);
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   */
-  private static void remoteThrowException() {
-    String s = "Test exception.  Please ignore.";
-    throw new BasicTestException(s);
-  }
-
-  private static void remoteBind(String name, String value) {
-    assertNotNull("name must not be null", name);
-    assertNotNull("value must not be null", value);
-    assertNotNull("bindings must not be null", bindings);
-
-    new JUnit3BasicDUnitTest("").getSystem(); // forces connection
-    bindings.setProperty(name, value);
-  }
-
-  private static void remoteValidateBind(String name, String expected) {
-    assertEquals(expected, bindings.getProperty(name));
-  }
-
-  private static class BasicTestException extends RuntimeException {
-    BasicTestException() {
-      this("Test exception.  Please ignore.");
-    }
-    BasicTestException(String s) {
-      super(s);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3GetDefaultDiskStoreNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3GetDefaultDiskStoreNameDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3GetDefaultDiskStoreNameDUnitTest.java
deleted file mode 100644
index e89f66b..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3GetDefaultDiskStoreNameDUnitTest.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.test.dunit.internal.tests;
-
-import static org.assertj.core.api.Assertions.*;
-
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.internal.JUnit3DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.experimental.categories.Category;
-
-@SuppressWarnings("serial")
-@Category(DistributedTest.class)
-public class JUnit3GetDefaultDiskStoreNameDUnitTest extends JUnit3DistributedTestCase {
-
-  public JUnit3GetDefaultDiskStoreNameDUnitTest(final String name) {
-    super(name);
-  }
-
-  public void testGetTestMethodName() {
-    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodName");
-    assertGetDefaultDiskStoreName(expected);
-  }
-
-  public void testGetTestMethodNameChanges() {
-    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodNameChanges");
-    assertGetDefaultDiskStoreName(expected);
-  }
-
-  public void testGetTestMethodNameInAllVMs() {
-    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodNameInAllVMs");
-    assertGetDefaultDiskStoreName(expected);
-
-    for (int vmIndex = 0; vmIndex < Host.getHost(0).getVMCount(); vmIndex++) {
-      String expectedInVM = createDefaultDiskStoreName(0, vmIndex, "testGetTestMethodNameInAllVMs");
-      Host.getHost(0).getVM(vmIndex).invoke(()->assertGetDefaultDiskStoreName(expectedInVM));
-    }
-  }
-
-  private void assertGetDefaultDiskStoreName(final String expected) {
-    assertThat(getDefaultDiskStoreName()).isEqualTo(expected);
-  }
-
-  private String createDefaultDiskStoreName(final int hostIndex, final int vmIndex, final String methodName) {
-    return "DiskStore-" + hostIndex + "-" + vmIndex + "-" + getClass().getCanonicalName() + "." + methodName;
-  }
-
-  private String getDefaultDiskStoreName() {
-    return GemFireCacheImpl.DEFAULT_DS_NAME; // TODO: not thread safe
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3GetTestMethodNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3GetTestMethodNameDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3GetTestMethodNameDUnitTest.java
deleted file mode 100644
index c77843d..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3GetTestMethodNameDUnitTest.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.test.dunit.internal.tests;
-
-import static org.assertj.core.api.Assertions.*;
-
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.internal.JUnit3DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.experimental.categories.Category;
-
-@SuppressWarnings("serial")
-@Category(DistributedTest.class)
-public class JUnit3GetTestMethodNameDUnitTest extends JUnit3DistributedTestCase {
-
-  public JUnit3GetTestMethodNameDUnitTest(final String name) {
-    super(name);
-  }
-
-  public void testGetTestMethodName() {
-    assertGetTestMethodName("testGetTestMethodName");
-  }
-
-  public void testGetTestMethodNameChanges() {
-    assertGetTestMethodName("testGetTestMethodNameChanges");
-  }
-
-  public void testGetTestMethodNameInAllVMs() {
-    assertGetTestMethodName("testGetTestMethodNameInAllVMs");
-
-    for (int vmIndex = 0; vmIndex < Host.getHost(0).getVMCount(); vmIndex++) {
-      Host.getHost(0).getVM(vmIndex).invoke(()->assertGetTestMethodName("testGetTestMethodNameInAllVMs"));
-    }
-  }
-
-  private void assertGetTestMethodName(final String expected) {
-    assertThat(getTestMethodName()).isEqualTo(expected);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3VMDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3VMDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3VMDUnitTest.java
deleted file mode 100644
index ac2ed7e..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit3VMDUnitTest.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.test.dunit.internal.tests;
-
-import java.io.Serializable;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.RMIException;
-import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.internal.JUnit3DistributedTestCase;
-
-/**
- * This class tests the functionality of the {@link VM} class.
- */
-public class JUnit3VMDUnitTest extends JUnit3DistributedTestCase {
-
-  private static final AtomicInteger COUNTER = new AtomicInteger();
-  private static final boolean BOOLEAN_VALUE = true;
-  private static final byte BYTE_VALUE = (byte) 40;
-  private static final long LONG_VALUE = 42L;
-  private static final String STRING_VALUE = "BLAH BLAH BLAH";
-
-  public JUnit3VMDUnitTest(String name) {
-    super(name);
-  }
-
-  public void testInvokeStaticBoolean() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(BOOLEAN_VALUE, (boolean) vm.invoke(() -> remoteBooleanMethod()));
-  }
-
-  public void testInvokeStaticByte() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(BYTE_VALUE, (byte) vm.invoke(() -> remoteByteMethod()));
-  }
-
-  public void testInvokeStaticLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(LONG_VALUE, (long) vm.invoke(() -> remoteLongMethod()));
-  }
-
-  public void testInvokeInstance() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(STRING_VALUE, vm.invoke(new ClassWithString(), "getString"));
-  }
-
-  public void testInvokeRunnableWithException() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(new InvokeRunnable());
-      fail("Should have thrown a BasicTestException");
-    } catch (RMIException ex) {
-      assertTrue(ex.getCause() instanceof BasicTestException);
-    }
-  }
-
-  public void testReturnValue() throws Exception {
-    final Host host = Host.getHost(0);
-    final VM vm = host.getVM(0);
-    // Assert class static invocation works
-    AsyncInvocation a1 = vm.invokeAsync(() -> getAndIncStaticCount());
-    a1.join();
-    assertEquals(new Integer(0), a1.getReturnValue());
-    // Assert class static invocation with args works
-    a1 = vm.invokeAsync(() -> incrementStaticCount(new Integer(2)));
-    a1.join();
-    assertEquals(new Integer(3), a1.getReturnValue());
-    // Assert that previous values are not returned when invoking method w/ no return val
-    a1 = vm.invokeAsync(() -> incStaticCount());
-    a1.join();
-    assertNull(a1.getReturnValue());
-    // Assert that previous null returns are over-written
-    a1 = vm.invokeAsync(() -> getAndIncStaticCount());
-    a1.join();
-    assertEquals(new Integer(4), a1.getReturnValue());
-
-    // Assert object method invocation works with zero arg method
-    final VMTestObject o = new VMTestObject(0);
-    a1 = vm.invokeAsync(o, "incrementAndGet", new Object[] {});
-    a1.join();
-    assertEquals(new Integer(1), a1.getReturnValue());
-    // Assert object method invocation works with no return
-    a1 = vm.invokeAsync(o, "set", new Object[] {new Integer(3)});
-    a1.join();
-    assertNull(a1.getReturnValue());
-  }
-
-  private static Integer getAndIncStaticCount() {
-    return new Integer(COUNTER.getAndIncrement());
-  }
-
-  private static Integer incrementStaticCount(Integer inc) {
-    return new Integer(COUNTER.addAndGet(inc.intValue()));
-  }
-
-  private static void incStaticCount() {
-    COUNTER.incrementAndGet();
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   */
-  private static byte remoteByteMethod() {
-    return BYTE_VALUE;
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   */
-  private static boolean remoteBooleanMethod() {
-    return BOOLEAN_VALUE;
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   */
-  private static long remoteLongMethod() {
-    return LONG_VALUE;
-  }
-
-  private static class ClassWithLong implements Serializable {
-    public long getLong() {
-      return LONG_VALUE;
-    }
-  }
-
-  private static class ClassWithByte implements Serializable {
-    public byte getByte() {
-      return BYTE_VALUE;
-    }
-  }
-
-  private static class InvokeRunnable implements SerializableRunnableIF {
-    public void run() {
-      throw new BasicTestException();
-    }
-  }
-
-  private static class ClassWithString implements Serializable {
-    public String getString() {
-      return STRING_VALUE;
-    }
-  }
-
-  private static class BasicTestException extends RuntimeException {
-    BasicTestException() {
-      this("Test exception.  Please ignore.");
-    }
-    BasicTestException(String s) {
-      super(s);
-    }
-  }
-
-  private static class VMTestObject implements Serializable {
-    private static final long serialVersionUID = 1L;
-    private final AtomicInteger val;
-    public VMTestObject(int init) {
-      this.val = new AtomicInteger(init);
-    }
-    public Integer get() {
-      return new Integer(this.val.get());
-    }
-    public Integer incrementAndGet() {
-      return new Integer(this.val.incrementAndGet());
-    }
-    public void set(Integer newVal) {
-      this.val.set(newVal.intValue());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4BasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4BasicDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4BasicDUnitTest.java
deleted file mode 100644
index 5cac3ba..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4BasicDUnitTest.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.test.dunit.internal.tests;
-
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-import static com.gemstone.gemfire.test.dunit.Invoke.*;
-
-import java.util.Properties;
-
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.DUnitEnv;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.RMIException;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * This class tests the basic functionality of the distributed unit
- * test framework.
- */
-@Category(DistributedTest.class)
-public class JUnit4BasicDUnitTest extends JUnit4DistributedTestCase {
-
-  private static Properties bindings;
-
-  @BeforeClass
-  public static void setUpJUnit4BasicDUnitTest() throws Exception {
-    invokeInEveryVM(() -> bindings = new Properties());
-  }
-
-  @AfterClass
-  public static void tearDownJUnit4BasicDUnitTest() {
-    invokeInEveryVM(() -> bindings = null);
-  }
-
-  @Test
-  public void testPreconditions() {
-    invokeInEveryVM(() -> assertNotNull("getUniqueName() must not return null", getUniqueName()));
-    invokeInEveryVM(() -> assertNotNull("bindings must not be null", bindings));
-  }
-
-  /**
-   * Tests how the Hydra framework handles an error
-   */
-  @Ignore
-  @Test
-  public void testDontCatchRemoteException() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    vm.invoke(() -> remoteThrowException());
-  }
-
-  @Test
-  public void testRemoteInvocationWithException() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(() -> remoteThrowException());
-      fail("Should have thrown a BasicTestException");
-
-    } catch (RMIException ex) {
-      assertTrue(ex.getCause() instanceof BasicTestException);
-    }
-  }
-
-  @Test
-  public void testInvokeWithLambda() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    int vm0Num = vm0.invoke(() -> DUnitEnv.get().getVMID());
-    int vm1Num = vm1.invoke(() -> DUnitEnv.get().getVMID());
-
-    assertEquals(0, vm0Num);
-    assertEquals(1, vm1Num);
-  }
-
-  @Test
-  public void testInvokeLambdaAsync() throws Throwable {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-
-    AsyncInvocation<Integer> async0 = vm0.invokeAsync(() -> DUnitEnv.get().getVMID());
-    int vm0num = async0.getResult();
-
-    assertEquals(0, vm0num);
-  }
-
-  @Test
-  public void testInvokeWithNamedLambda() {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    int vm0Num = vm0.invoke("getVMID", () -> DUnitEnv.get().getVMID());
-    int vm1Num = vm1.invoke("getVMID", () -> DUnitEnv.get().getVMID());
-
-    assertEquals(0, vm0Num);
-    assertEquals(1, vm1Num);
-  }
-
-  @Test
-  public void testInvokeNamedLambdaAsync() throws Throwable {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-
-    AsyncInvocation<Integer> async0 = vm0.invokeAsync("getVMID", () -> DUnitEnv.get().getVMID());
-    int vm0num = async0.getResult();
-
-    assertEquals(0, vm0num);
-  }
-
-  @Ignore("Test was never implemented")
-  @Test
-  public void testRemoteInvocationBoolean() {
-  }
-
-  @Test
-  public void testRemoteInvokeAsync() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    String name = this.getUniqueName();
-    String value = "Hello";
-
-    AsyncInvocation ai =
-            vm.invokeAsync(() -> this.remoteBind( name, value ));
-    ai.join();
-    // TODO shouldn't we call fail() here?
-    if (ai.exceptionOccurred()) {
-      fail("remoteBind failed", ai.getException());
-    }
-
-    ai = vm.invokeAsync(() -> this.remoteValidateBind(name, value ));
-    ai.join();
-    if (ai.exceptionOccurred()) {
-      fail("remoteValidateBind failed", ai.getException());
-    }
-  }
-
-  @Test
-  public void testRemoteInvokeAsyncWithException() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-
-    AsyncInvocation ai = vm.invokeAsync(() -> this.remoteThrowException());
-    ai.join();
-    assertTrue(ai.exceptionOccurred());
-    Throwable ex = ai.getException();
-    assertTrue(ex instanceof BasicTestException);
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   */
-  private static void remoteThrowException() {
-    String s = "Test exception.  Please ignore.";
-    throw new BasicTestException(s);
-  }
-
-  private static void remoteBind(String name, String value) {
-    assertNotNull("name must not be null", name);
-    assertNotNull("value must not be null", value);
-    assertNotNull("bindings must not be null", bindings);
-
-    new JUnit4BasicDUnitTest().getSystem(); // forces connection
-    bindings.setProperty(name, value);
-  }
-
-  private static void remoteValidateBind(String name, String expected) {
-    assertEquals(expected, bindings.getProperty(name));
-  }
-
-  private static class BasicTestException extends RuntimeException {
-    BasicTestException() {
-      this("Test exception.  Please ignore.");
-    }
-    BasicTestException(String s) {
-      super(s);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4GetDefaultDiskStoreNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4GetDefaultDiskStoreNameDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4GetDefaultDiskStoreNameDUnitTest.java
deleted file mode 100644
index 7f99bc1..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4GetDefaultDiskStoreNameDUnitTest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.test.dunit.internal.tests;
-
-import static org.assertj.core.api.Assertions.*;
-
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(DistributedTest.class)
-public class JUnit4GetDefaultDiskStoreNameDUnitTest extends JUnit4DistributedTestCase {
-
-  @Test
-  public void testGetTestMethodName() {
-    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodName");
-    assertGetDefaultDiskStoreName(expected);
-  }
-
-  @Test
-  public void testGetTestMethodNameChanges() {
-    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodNameChanges");
-    assertGetDefaultDiskStoreName(expected);
-  }
-
-  @Test
-  public void testGetTestMethodNameInAllVMs() {
-    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodNameInAllVMs");
-    assertGetDefaultDiskStoreName(expected);
-
-    for (int vmIndex = 0; vmIndex < Host.getHost(0).getVMCount(); vmIndex++) {
-      String expectedInVM = createDefaultDiskStoreName(0, vmIndex, "testGetTestMethodNameInAllVMs");
-      Host.getHost(0).getVM(vmIndex).invoke(()->assertGetDefaultDiskStoreName(expectedInVM));
-    }
-  }
-
-  private void assertGetDefaultDiskStoreName(final String expected) {
-    assertThat(getDefaultDiskStoreName()).isEqualTo(expected);
-  }
-
-  private String createDefaultDiskStoreName(final int hostIndex, final int vmIndex, final String methodName) {
-    return "DiskStore-" + hostIndex + "-" + vmIndex + "-" + getClass().getCanonicalName() + "." + methodName;
-  }
-
-  private String getDefaultDiskStoreName() {
-    return GemFireCacheImpl.DEFAULT_DS_NAME; // TODO: not thread safe
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4GetTestMethodNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4GetTestMethodNameDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4GetTestMethodNameDUnitTest.java
deleted file mode 100644
index 93b2e86..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4GetTestMethodNameDUnitTest.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.test.dunit.internal.tests;
-
-import static org.assertj.core.api.Assertions.*;
-
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(DistributedTest.class)
-public class JUnit4GetTestMethodNameDUnitTest extends JUnit4DistributedTestCase {
-
-  @Test
-  public void testGetTestMethodName() {
-    assertGetTestMethodName("testGetTestMethodName");
-  }
-
-  @Test
-  public void testGetTestMethodNameChanges() {
-    assertGetTestMethodName("testGetTestMethodNameChanges");
-  }
-
-  @Test
-  public void testGetTestMethodNameInAllVMs() {
-    assertGetTestMethodName("testGetTestMethodNameInAllVMs");
-
-    for (int vmIndex = 0; vmIndex < Host.getHost(0).getVMCount(); vmIndex++) {
-      Host.getHost(0).getVM(vmIndex).invoke(()->assertGetTestMethodName("testGetTestMethodNameInAllVMs"));
-    }
-  }
-
-  private void assertGetTestMethodName(final String expected) {
-    assertThat(getTestMethodName()).isEqualTo(expected);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4VMDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4VMDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4VMDUnitTest.java
deleted file mode 100644
index c9d9d3c..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/tests/JUnit4VMDUnitTest.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.test.dunit.internal.tests;
-
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-
-import java.io.Serializable;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.RMIException;
-import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * This class tests the functionality of the {@link VM} class.
- */
-@Category(DistributedTest.class)
-public class JUnit4VMDUnitTest extends JUnit4DistributedTestCase {
-
-  private static final AtomicInteger COUNTER = new AtomicInteger();
-  private static final boolean BOOLEAN_VALUE = true;
-  private static final byte BYTE_VALUE = (byte) 40;
-  private static final long LONG_VALUE = 42L;
-  private static final String STRING_VALUE = "BLAH BLAH BLAH";
-
-  @Test
-  public void testInvokeStaticBoolean() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(BOOLEAN_VALUE, (boolean) vm.invoke(() -> remoteBooleanMethod()));
-  }
-
-  @Test
-  public void testInvokeStaticByte() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(BYTE_VALUE, (byte) vm.invoke(() -> remoteByteMethod()));
-  }
-
-  @Test
-  public void testInvokeStaticLong() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(LONG_VALUE, (long) vm.invoke(() -> remoteLongMethod()));
-  }
-
-  @Test
-  public void testInvokeInstance() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    assertEquals(STRING_VALUE, vm.invoke(new ClassWithString(), "getString"));
-  }
-
-  @Test
-  public void testInvokeRunnableWithException() {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(new InvokeRunnable());
-      fail("Should have thrown a BasicTestException");
-    } catch (RMIException ex) {
-      assertTrue(ex.getCause() instanceof BasicTestException);
-    }
-  }
-
-  @Test
-  public void testReturnValue() throws Exception {
-    final Host host = Host.getHost(0);
-    final VM vm = host.getVM(0);
-    // Assert class static invocation works
-    AsyncInvocation a1 = vm.invokeAsync(() -> getAndIncStaticCount());
-    a1.join();
-    assertEquals(new Integer(0), a1.getReturnValue());
-    // Assert class static invocation with args works
-    a1 = vm.invokeAsync(() -> incrementStaticCount(new Integer(2)));
-    a1.join();
-    assertEquals(new Integer(3), a1.getReturnValue());
-    // Assert that previous values are not returned when invoking method w/ no return val
-    a1 = vm.invokeAsync(() -> incStaticCount());
-    a1.join();
-    assertNull(a1.getReturnValue());
-    // Assert that previous null returns are over-written
-    a1 = vm.invokeAsync(() -> getAndIncStaticCount());
-    a1.join();
-    assertEquals(new Integer(4), a1.getReturnValue());
-
-    // Assert object method invocation works with zero arg method
-    final VMTestObject o = new VMTestObject(0);
-    a1 = vm.invokeAsync(o, "incrementAndGet", new Object[] {});
-    a1.join();
-    assertEquals(new Integer(1), a1.getReturnValue());
-    // Assert object method invocation works with no return
-    a1 = vm.invokeAsync(o, "set", new Object[] {new Integer(3)});
-    a1.join();
-    assertNull(a1.getReturnValue());
-  }
-
-  private static Integer getAndIncStaticCount() {
-    return new Integer(COUNTER.getAndIncrement());
-  }
-
-  private static Integer incrementStaticCount(Integer inc) {
-    return new Integer(COUNTER.addAndGet(inc.intValue()));
-  }
-
-  private static void incStaticCount() {
-    COUNTER.incrementAndGet();
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   */
-  private static byte remoteByteMethod() {
-    return BYTE_VALUE;
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   */
-  private static boolean remoteBooleanMethod() {
-    return BOOLEAN_VALUE;
-  }
-
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   */
-  private static long remoteLongMethod() {
-    return LONG_VALUE;
-  }
-
-  private static class ClassWithLong implements Serializable {
-    public long getLong() {
-      return LONG_VALUE;
-    }
-  }
-
-  private static class ClassWithByte implements Serializable {
-    public byte getByte() {
-      return BYTE_VALUE;
-    }
-  }
-
-  private static class InvokeRunnable implements SerializableRunnableIF {
-    public void run() {
-      throw new BasicTestException();
-    }
-  }
-
-  private static class ClassWithString implements Serializable {
-    public String getString() {
-      return STRING_VALUE;
-    }
-  }
-
-  private static class BasicTestException extends RuntimeException {
-    BasicTestException() {
-      this("Test exception.  Please ignore.");
-    }
-    BasicTestException(String s) {
-      super(s);
-    }
-  }
-
-  private static class VMTestObject implements Serializable {
-    private static final long serialVersionUID = 1L;
-    private final AtomicInteger val;
-    public VMTestObject(int init) {
-      this.val = new AtomicInteger(init);
-    }
-    public Integer get() {
-      return new Integer(this.val.get());
-    }
-    public Integer incrementAndGet() {
-      return new Integer(this.val.incrementAndGet());
-    }
-    public void set(Integer newVal) {
-      this.val.set(newVal.intValue());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java
index 7f95fad..73e813d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/DistributedRestoreSystemProperties.java
@@ -16,8 +16,7 @@
  */
 package com.gemstone.gemfire.test.dunit.rules;
 
-import static java.lang.System.getProperties;
-import static java.lang.System.setProperties;
+import static java.lang.System.*;
 
 import java.util.Properties;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java
index 98dbc2f..789af38 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/rules/RemoteInvoker.java
@@ -16,8 +16,7 @@
  */
 package com.gemstone.gemfire.test.dunit.rules;
 
-import static com.gemstone.gemfire.test.dunit.Invoke.invokeInEveryVM;
-import static com.gemstone.gemfire.test.dunit.Invoke.invokeInLocator;
+import static com.gemstone.gemfire.test.dunit.Invoke.*;
 
 import java.io.Serializable;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
index 06c3196..5301ffe 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ChildVM.java
@@ -16,11 +16,10 @@
  */
 package com.gemstone.gemfire.test.dunit.standalone;
 
-import hydra.HydraRuntimeException;
-import hydra.Log;
-
 import java.rmi.Naming;
 
+import hydra.HydraRuntimeException;
+import hydra.Log;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.internal.OSProcess;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
index 216971a..be459b9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
@@ -16,9 +16,6 @@
  */
 package com.gemstone.gemfire.test.dunit.standalone;
 
-import hydra.Log;
-import hydra.MethExecutorResult;
-
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -41,6 +38,10 @@ import java.rmi.server.UnicastRemoteObject;
 import java.util.List;
 import java.util.Properties;
 
+import batterytest.greplogs.ExpectedStrings;
+import batterytest.greplogs.LogConsumer;
+import hydra.Log;
+import hydra.MethExecutorResult;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.core.LoggerContext;
@@ -49,9 +50,6 @@ import org.apache.logging.log4j.core.config.LoggerConfig;
 import org.apache.logging.log4j.core.layout.PatternLayout;
 import org.junit.Assert;
 
-import batterytest.greplogs.ExpectedStrings;
-import batterytest.greplogs.LogConsumer;
-
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
index 81c895e..21f35ac 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVM.java
@@ -20,13 +20,12 @@ import java.rmi.RemoteException;
 import java.rmi.server.UnicastRemoteObject;
 import java.util.concurrent.TimeUnit;
 
+import hydra.MethExecutor;
+import hydra.MethExecutorResult;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.internal.logging.LogService;
 
-import hydra.MethExecutor;
-import hydra.MethExecutorResult;
-
 /**
  *
  */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
index 849e2f2..57bbee2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/RemoteDUnitVMIF.java
@@ -16,11 +16,11 @@
  */
 package com.gemstone.gemfire.test.dunit.standalone;
 
-import hydra.MethExecutorResult;
-
 import java.rmi.Remote;
 import java.rmi.RemoteException;
 
+import hydra.MethExecutorResult;
+
 public interface RemoteDUnitVMIF extends Remote {
 
   MethExecutorResult executeMethodOnObject(Object o, String methodName) throws RemoteException;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
index b10f610..17afcc6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
@@ -16,11 +16,14 @@
  */
 package com.gemstone.gemfire.test.dunit.tests;
 
-import static com.gemstone.gemfire.test.dunit.Invoke.invokeInEveryVM;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Invoke.*;
+import static com.googlecode.catchexception.CatchException.*;
+import static com.googlecode.catchexception.throwable.CatchThrowable.*;
+import static org.hamcrest.Matchers.*;
 
 import java.util.Properties;
 
-import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.DUnitEnv;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
@@ -32,10 +35,16 @@ import com.gemstone.gemfire.test.dunit.VM;
  * This class tests the basic functionality of the distributed unit
  * test framework.
  */
+@SuppressWarnings("unused")
 public class BasicDUnitTest extends DistributedTestCase {
 
+  private static final String MESSAGE_FOR_remoteThrowException = "Test exception.  Please ignore.";
+
   private static Properties bindings;
 
+  private VM vm0;
+  private VM vm1;
+
   public BasicDUnitTest(String name) {
     super(name);
   }
@@ -44,6 +53,8 @@ public class BasicDUnitTest extends DistributedTestCase {
   public final void postSetUp() throws Exception {
     bindings = new Properties();
     invokeInEveryVM(() -> bindings = new Properties());
+    this.vm0 = Host.getHost(0).getVM(0);
+    this.vm1 = Host.getHost(0).getVM(1);
   }
 
   @Override
@@ -52,117 +63,121 @@ public class BasicDUnitTest extends DistributedTestCase {
     invokeInEveryVM(() -> bindings = null);
   }
 
-  public void testPreconditions() {
-    invokeInEveryVM(() -> assertNotNull("getUniqueName() must not return null", getUniqueName()));
-    invokeInEveryVM(() -> assertNotNull("bindings must not be null", bindings));
+  public void testPreconditions() throws Exception {
+    invokeInEveryVM(() -> assertThat("getUniqueName() must not return null", getUniqueName(), notNullValue()));
+    invokeInEveryVM(() -> assertThat("bindings must not be null", bindings, notNullValue()));
   }
 
-  /**
-   * Tests how the Hydra framework handles an error
-   */
-  public void ignore_testDontCatchRemoteException() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    vm.invoke(() -> remoteThrowException());
+  public void testInvokeOnClassTargetWithEmptyArgs() throws Exception {
+    assertThat(this.vm0.invoke(BasicDUnitTest.class, "booleanValue", new Object[] {}), is(true));
+  }
+  public void testInvokeOnObjectTargetWithEmptyArgs() throws Exception {
+    assertThat(this.vm0.invoke(new BasicDUnitTest(""), "booleanValue", new Object[] {}), is(true));
+  }
+  public void testInvokeAsyncOnClassTargetWithEmptyArgs() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(BasicDUnitTest.class, "booleanValue", new Object[] {}).join();
+    assertThat(async.getResult(), is(true));
+  }
+  public void testInvokeAsyncOnObjectTargetWithEmptyArgs() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(new BasicDUnitTest(""), "booleanValue", new Object[] {}).join();
+    assertThat(async.getResult(), is(true));
   }
 
-  public void testRemoteInvocationWithException() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    try {
-      vm.invoke(() -> remoteThrowException());
-      fail("Should have thrown a BasicTestException");
-
-    } catch (RMIException ex) {
-      assertTrue(ex.getCause() instanceof BasicTestException);
-    }
+  public void testInvokeOnClassTargetWithNullArgs() throws Exception {
+    assertThat(this.vm0.invoke(BasicDUnitTest.class, "booleanValue", null), is(true));
+  }
+  public void testInvokeOnObjectTargetWithNullArgs() throws Exception {
+    assertThat(this.vm0.invoke(new BasicDUnitTest(""), "booleanValue", null), is(true));
+  }
+  public void testInvokeAsyncOnClassTargetWithNullArgs() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(BasicDUnitTest.class, "booleanValue", null).join();
+    assertThat(async.getResult(), is(true));
+  }
+  public void testInvokeAsyncOnObjectTargetWithNullArgs() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(new BasicDUnitTest(""), "booleanValue", null).join();
+    assertThat(async.getResult(), is(true));
   }
 
-  public void testInvokeWithLambda() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
+  public void testRemoteInvocationWithException() throws Exception {
+    catchException(this.vm0).invoke(() -> remoteThrowException());
 
-    int vm0Num = vm0.invoke(() -> DUnitEnv.get().getVMID());
-    int vm1Num = vm1.invoke(() -> DUnitEnv.get().getVMID());
+    assertThat(caughtException(), instanceOf(RMIException.class));
+    assertThat(caughtException().getCause(), notNullValue());
+    assertThat(caughtException().getCause(), instanceOf(BasicTestException.class));
+    assertThat(caughtException().getCause().getMessage(), is(MESSAGE_FOR_remoteThrowException));
+  }
 
-    assertEquals(0, vm0Num);
-    assertEquals(1, vm1Num);
+  public void testInvokeWithLambda() throws Exception {
+    assertThat(this.vm0.invoke(() -> DUnitEnv.get().getVMID()), is(0));
+    assertThat(this.vm1.invoke(() -> DUnitEnv.get().getVMID()), is(1));
   }
 
   public void testInvokeLambdaAsync() throws Throwable {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-
-    AsyncInvocation<Integer> async0 = vm0.invokeAsync(() -> DUnitEnv.get().getVMID());
-    int vm0num = async0.getResult();
-
-    assertEquals(0, vm0num);
+    assertThat(this.vm0.invokeAsync(() -> DUnitEnv.get().getVMID()).getResult(), is(0));
   }
 
   public void testInvokeWithNamedLambda() {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    int vm0Num = vm0.invoke("getVMID", () -> DUnitEnv.get().getVMID());
-    int vm1Num = vm1.invoke("getVMID", () -> DUnitEnv.get().getVMID());
-
-    assertEquals(0, vm0Num);
-    assertEquals(1, vm1Num);
+    assertThat(this.vm0.invoke("getVMID", () -> DUnitEnv.get().getVMID()), is(0));
+    assertThat(this.vm1.invoke("getVMID", () -> DUnitEnv.get().getVMID()), is(1));
   }
 
   public void testInvokeNamedLambdaAsync() throws Throwable {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
+    assertThat(this.vm0.invokeAsync("getVMID", () -> DUnitEnv.get().getVMID()).getResult(), is(0));
+  }
 
-    AsyncInvocation<Integer> async0 = vm0.invokeAsync("getVMID", () -> DUnitEnv.get().getVMID());
-    int vm0num = async0.getResult();
+  public void testRemoteInvokeAsync() throws Exception {
+    String name = getUniqueName();
+    String value = "Hello";
 
-    assertEquals(0, vm0num);
+    this.vm0.invokeAsync(() -> remoteBind(name, value)).join().checkException();
+    this.vm0.invokeAsync(() -> remoteValidateBind(name, value )).join().checkException();
   }
 
-  // Test was never implemented
-  public void ignore_testRemoteInvocationBoolean() {
+  public void testRemoteInvokeAsyncWithException() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(() -> remoteThrowException()).join();
+
+    assertThat(async.exceptionOccurred(), is(true));
+    assertThat(async.getException(), instanceOf(BasicTestException.class));
+
+    catchThrowable(async).checkException();
+
+    assertThat(caughtThrowable(), instanceOf(AssertionError.class));
+    assertThat(caughtThrowable().getCause(), notNullValue());
+    assertThat(caughtThrowable().getCause(), instanceOf(BasicTestException.class));
+    assertThat(caughtThrowable().getCause().getMessage(), is(MESSAGE_FOR_remoteThrowException));
   }
 
-  public void testRemoteInvokeAsync() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
-    String name = getUniqueName();
-    String value = "Hello";
+  public void testInvokeNamedRunnableLambdaAsync() throws Exception {
+    catchThrowable(this.vm0.invokeAsync("throwSomething", () -> throwException()).join()).checkException();
 
-    AsyncInvocation ai = vm.invokeAsync(() -> remoteBind(name, value));
-    ai.join();
-    // TODO shouldn't we call fail() here?
-    if (ai.exceptionOccurred()) {
-      Assert.fail("remoteBind failed", ai.getException());
-    }
+    assertThat(caughtThrowable(), notNullValue());
+    assertThat(caughtThrowable().getCause(), notNullValue());
+    assertThat(caughtThrowable().getCause(), instanceOf(BasicDUnitException.class));
+  }
 
-    ai = vm.invokeAsync(() -> remoteValidateBind(name, value ));
-    ai.join();
-    if (ai.exceptionOccurred()) {
-      Assert.fail("remoteValidateBind failed", ai.getException());
-    }
+  public void testInvokeNamedRunnableLambda() throws Exception {
+    catchException(this.vm0).invoke("throwSomething", () -> throwException());
+
+    assertThat(caughtException(), notNullValue());
+    assertThat(caughtException().getCause(), notNullValue());
+    assertThat(caughtException().getCause(), instanceOf(BasicDUnitException.class));
+    assertThat(caughtException().getCause().getMessage(), nullValue());
   }
 
-  public void testRemoteInvokeAsyncWithException() throws Exception {
-    Host host = Host.getHost(0);
-    VM vm = host.getVM(0);
+  private static boolean booleanValue() { // invoked by reflection
+    return true;
+  }
 
-    AsyncInvocation ai = vm.invokeAsync(() -> remoteThrowException());
-    ai.join();
-    assertTrue(ai.exceptionOccurred());
-    Throwable ex = ai.getException();
-    assertTrue(ex instanceof BasicTestException);
+  private static boolean booleanValue(final boolean value) { // invoked by reflection
+    return value;
   }
 
-  /**
-   * Accessed via reflection.  DO NOT REMOVE
-   */
   private static void remoteThrowException() {
-    String s = "Test exception.  Please ignore.";
-    throw new BasicTestException(s);
+    throw new BasicTestException(MESSAGE_FOR_remoteThrowException);
+  }
+
+  private static void throwException() throws BasicDUnitException {
+    throw new BasicDUnitException();
   }
 
   private static void remoteBind(String name, String value) {
@@ -187,45 +202,8 @@ public class BasicDUnitTest extends DistributedTestCase {
     }
   }
 
-  static class BasicDUnitException extends RuntimeException {
+  private static class BasicDUnitException extends RuntimeException {
     public BasicDUnitException() {
     }
   }
-
-  public static void throwException() throws BasicDUnitException {
-    throw new BasicDUnitException();
-  }
-
-  public void testInvokeNamedRunnableLambdaAsync() throws Throwable {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-
-    AsyncInvocation<Integer> async0 = vm0.invokeAsync("throwSomething", () -> BasicDUnitTest.throwException());
-    try {
-      async0.getResult();
-      throw new Error("expected an exception to be thrown");
-    } catch (Exception e) {
-      Throwable cause = e.getCause();
-      if (cause == null) {
-        throw new Error("expected an exception with a cause to be thrown", e);
-      }
-      if ( !(cause.getCause() instanceof BasicDUnitException) ) {
-        throw new Error("expected a BasicDUnitException to be thrown", e.getCause());
-      }
-    }
-  }
-
-  public void testInvokeNamedRunnableLambda() throws Throwable {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-
-    try {
-      vm0.invoke("throwSomething", () -> BasicDUnitTest.throwException());
-      throw new Error("expected an exception to be thrown");
-    } catch (Exception e) {
-      if ( !(e.getCause() instanceof BasicDUnitException) ) {
-        throw new Error("expected a BasicDUnitException to be thrown", e.getCause());
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4BasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4BasicDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4BasicDUnitTest.java
new file mode 100644
index 0000000..01ba152
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4BasicDUnitTest.java
@@ -0,0 +1,234 @@
+/*
+ * 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.test.dunit.tests;
+
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Invoke.*;
+import static com.googlecode.catchexception.CatchException.*;
+import static com.googlecode.catchexception.throwable.CatchThrowable.*;
+import static org.hamcrest.Matchers.*;
+
+import java.util.Properties;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.dunit.DUnitEnv;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.RMIException;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+/**
+ * This class tests the basic functionality of the distributed unit
+ * test framework.
+ *
+ * @see JUnit4BasicDUnitTest
+ */
+@Category(DistributedTest.class)
+@SuppressWarnings("unused")
+public class JUnit4BasicDUnitTest extends JUnit4DistributedTestCase {
+
+  private static final String MESSAGE_FOR_remoteThrowException = "Test exception.  Please ignore.";
+
+  private static Properties bindings;
+
+  private VM vm0;
+  private VM vm1;
+
+  @BeforeClass
+  public static void setUpJUnit4BasicDUnitTest() throws Exception {
+    invokeInEveryVM(() -> bindings = new Properties());
+  }
+
+  @Override
+  public final void postSetUp() throws Exception {
+    this.vm0 = Host.getHost(0).getVM(0);
+    this.vm1 = Host.getHost(0).getVM(1);
+  }
+
+  @AfterClass
+  public static void tearDownJUnit4BasicDUnitTest() {
+    invokeInEveryVM(() -> bindings = null);
+  }
+
+  @Test
+  public void testPreconditions() {
+    invokeInEveryVM(() -> assertThat("getUniqueName() must not return null", getUniqueName(), notNullValue()));
+    invokeInEveryVM(() -> assertThat("bindings must not be null", bindings, notNullValue()));
+  }
+
+  @Test
+  public void testInvokeOnClassTargetWithEmptyArgs() throws Exception {
+    assertThat(this.vm0.invoke(JUnit4BasicDUnitTest.class, "booleanValue", new Object[] {}), is(true));
+  }
+  @Test
+  public void testInvokeOnObjectTargetWithEmptyArgs() throws Exception {
+    assertThat(this.vm0.invoke(new JUnit4BasicDUnitTest(), "booleanValue", new Object[] {}), is(true));
+  }
+  @Test
+  public void testInvokeAsyncOnClassTargetWithEmptyArgs() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(JUnit4BasicDUnitTest.class, "booleanValue", new Object[] {}).join();
+    assertThat(async.getResult(), is(true));
+  }
+  @Test
+  public void testInvokeAsyncOnObjectTargetWithEmptyArgs() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(new JUnit4BasicDUnitTest(), "booleanValue", new Object[] {}).join();
+    assertThat(async.getResult(), is(true));
+  }
+
+  @Test
+  public void testInvokeOnClassTargetWithNullArgs() throws Exception {
+    assertThat(this.vm0.invoke(JUnit4BasicDUnitTest.class, "booleanValue", null), is(true));
+  }
+  @Test
+  public void testInvokeOnObjectTargetWithNullArgs() throws Exception {
+    assertThat(this.vm0.invoke(new JUnit4BasicDUnitTest(), "booleanValue", null), is(true));
+  }
+  @Test
+  public void testInvokeAsyncOnClassTargetWithNullArgs() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(JUnit4BasicDUnitTest.class, "booleanValue", null).join();
+    assertThat(async.getResult(), is(true));
+  }
+  @Test
+  public void testInvokeAsyncOnObjectTargetWithNullArgs() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(new JUnit4BasicDUnitTest(), "booleanValue", null).join();
+    assertThat(async.getResult(), is(true));
+  }
+
+  @Test
+  public void testRemoteInvocationWithException() throws Exception {
+    catchException(this.vm0).invoke(() -> remoteThrowException());
+
+    assertThat(caughtException(), instanceOf(RMIException.class));
+    assertThat(caughtException().getCause(), notNullValue());
+    assertThat(caughtException().getCause(), instanceOf(BasicTestException.class));
+    assertThat(caughtException().getCause().getMessage(), is(MESSAGE_FOR_remoteThrowException));
+  }
+
+  @Test
+  public void testInvokeWithLambda() throws Exception {
+    assertThat(this.vm0.invoke(() -> DUnitEnv.get().getVMID()), is(0));
+    assertThat(this.vm1.invoke(() -> DUnitEnv.get().getVMID()), is(1));
+  }
+
+  @Test
+  public void testInvokeLambdaAsync() throws Throwable {
+    assertThat(this.vm0.invokeAsync(() -> DUnitEnv.get().getVMID()).getResult(), is(0));
+  }
+
+  @Test
+  public void testInvokeWithNamedLambda() {
+    assertThat(this.vm0.invoke("getVMID", () -> DUnitEnv.get().getVMID()), is(0));
+    assertThat(this.vm1.invoke("getVMID", () -> DUnitEnv.get().getVMID()), is(1));
+  }
+
+  @Test
+  public void testInvokeNamedLambdaAsync() throws Throwable {
+    assertThat(this.vm0.invokeAsync("getVMID", () -> DUnitEnv.get().getVMID()).getResult(), is(0));
+  }
+
+  @Test
+  public void testRemoteInvokeAsync() throws Exception {
+    String name = getUniqueName();
+    String value = "Hello";
+
+    this.vm0.invokeAsync(() -> remoteBind(name, value)).join().checkException();
+    this.vm0.invokeAsync(() -> remoteValidateBind(name, value )).join().checkException();
+  }
+
+  @Test
+  public void testRemoteInvokeAsyncWithException() throws Exception {
+    AsyncInvocation<?> async = this.vm0.invokeAsync(() -> remoteThrowException()).join();
+
+    assertThat(async.exceptionOccurred(), is(true));
+    assertThat(async.getException(), instanceOf(BasicTestException.class));
+
+    catchThrowable(async).checkException();
+
+    assertThat(caughtThrowable(), instanceOf(AssertionError.class));
+    assertThat(caughtThrowable().getCause(), notNullValue());
+    assertThat(caughtThrowable().getCause(), instanceOf(BasicTestException.class));
+    assertThat(caughtThrowable().getCause().getMessage(), is(MESSAGE_FOR_remoteThrowException));
+  }
+
+  @Test
+  public void testInvokeNamedRunnableLambdaAsync() throws Exception {
+    catchThrowable(this.vm0.invokeAsync("throwSomething", () -> throwException()).join()).checkException();
+
+    assertThat(caughtThrowable(), notNullValue());
+    assertThat(caughtThrowable().getCause(), notNullValue());
+    assertThat(caughtThrowable().getCause(), instanceOf(BasicDUnitException.class));
+  }
+
+  @Test
+  public void testInvokeNamedRunnableLambda() throws Exception {
+    catchException(this.vm0).invoke("throwSomething", () -> throwException());
+
+    assertThat(caughtException(), notNullValue());
+    assertThat(caughtException().getCause(), notNullValue());
+    assertThat(caughtException().getCause(), instanceOf(BasicDUnitException.class));
+    assertThat(caughtException().getCause().getMessage(), nullValue());
+  }
+
+  private static boolean booleanValue() { // invoked by reflection
+    return true;
+  }
+
+  private static boolean booleanValue(final boolean value) { // invoked by reflection
+    return value;
+  }
+
+  private static void remoteThrowException() {
+    throw new BasicTestException(MESSAGE_FOR_remoteThrowException);
+  }
+
+  private static void throwException() throws BasicDUnitException {
+    throw new BasicDUnitException();
+  }
+
+  private static void remoteBind(String name, String value) {
+    assertNotNull("name must not be null", name);
+    assertNotNull("value must not be null", value);
+    assertNotNull("bindings must not be null", bindings);
+
+    new JUnit4BasicDUnitTest().getSystem(); // forces connection
+    bindings.setProperty(name, value);
+  }
+
+  private static void remoteValidateBind(String name, String expected) {
+    assertEquals(expected, bindings.getProperty(name));
+  }
+
+  private static class BasicTestException extends RuntimeException {
+    BasicTestException() {
+      this("Test exception.  Please ignore.");
+    }
+    BasicTestException(String s) {
+      super(s);
+    }
+  }
+
+  private static class BasicDUnitException extends RuntimeException {
+    public BasicDUnitException() {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4GetDefaultDiskStoreNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4GetDefaultDiskStoreNameDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4GetDefaultDiskStoreNameDUnitTest.java
new file mode 100644
index 0000000..cab5559
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4GetDefaultDiskStoreNameDUnitTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.test.dunit.tests;
+
+import static org.assertj.core.api.Assertions.*;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class JUnit4GetDefaultDiskStoreNameDUnitTest extends JUnit4DistributedTestCase {
+
+  @Test
+  public void testGetTestMethodName() {
+    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodName");
+    assertGetDefaultDiskStoreName(expected);
+  }
+
+  @Test
+  public void testGetTestMethodNameChanges() {
+    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodNameChanges");
+    assertGetDefaultDiskStoreName(expected);
+  }
+
+  @Test
+  public void testGetTestMethodNameInAllVMs() {
+    String expected = createDefaultDiskStoreName(0, -1, "testGetTestMethodNameInAllVMs");
+    assertGetDefaultDiskStoreName(expected);
+
+    for (int vmIndex = 0; vmIndex < Host.getHost(0).getVMCount(); vmIndex++) {
+      String expectedInVM = createDefaultDiskStoreName(0, vmIndex, "testGetTestMethodNameInAllVMs");
+      Host.getHost(0).getVM(vmIndex).invoke(()->assertGetDefaultDiskStoreName(expectedInVM));
+    }
+  }
+
+  private void assertGetDefaultDiskStoreName(final String expected) {
+    assertThat(getDefaultDiskStoreName()).isEqualTo(expected);
+  }
+
+  private String createDefaultDiskStoreName(final int hostIndex, final int vmIndex, final String methodName) {
+    return "DiskStore-" + hostIndex + "-" + vmIndex + "-" + getClass().getCanonicalName() + "." + methodName;
+  }
+
+  private String getDefaultDiskStoreName() {
+    return GemFireCacheImpl.DEFAULT_DS_NAME; // TODO: not thread safe
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4GetTestMethodNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4GetTestMethodNameDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4GetTestMethodNameDUnitTest.java
new file mode 100644
index 0000000..a4a9b49
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4GetTestMethodNameDUnitTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.test.dunit.tests;
+
+import static org.assertj.core.api.Assertions.*;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class JUnit4GetTestMethodNameDUnitTest extends JUnit4DistributedTestCase {
+
+  @Test
+  public void testGetTestMethodName() {
+    assertGetTestMethodName("testGetTestMethodName");
+  }
+
+  @Test
+  public void testGetTestMethodNameChanges() {
+    assertGetTestMethodName("testGetTestMethodNameChanges");
+  }
+
+  @Test
+  public void testGetTestMethodNameInAllVMs() {
+    assertGetTestMethodName("testGetTestMethodNameInAllVMs");
+
+    for (int vmIndex = 0; vmIndex < Host.getHost(0).getVMCount(); vmIndex++) {
+      Host.getHost(0).getVM(vmIndex).invoke(()->assertGetTestMethodName("testGetTestMethodNameInAllVMs"));
+    }
+  }
+
+  private void assertGetTestMethodName(final String expected) {
+    assertThat(getTestMethodName()).isEqualTo(expected);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
new file mode 100644
index 0000000..b03c42a
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.test.dunit.tests;
+
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Invoke.*;
+
+import java.util.Properties;
+
+import org.junit.Test;
+
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+
+/**
+ * Verifies that overriding {@code getDistributedSystemProperties} results
+ * in {@code disconnectAllFromDS} during tear down.
+ */
+public class JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest extends JUnit4DistributedTestCase {
+
+  @Override
+  public final void preTearDownAssertions() throws Exception {
+    invokeInEveryVM(() -> assertNotNull(basicGetSystem()));
+  }
+
+  @Override
+  public final void postTearDownAssertions() throws Exception {
+    invokeInEveryVM(() -> assertNull(basicGetSystem()));
+  }
+
+  @Override
+  public final Properties getDistributedSystemProperties() {
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    return props;
+  }
+
+  @Test
+  public void testDisconnects() throws Exception {
+    invokeInEveryVM(() -> assertFalse(getDistributedSystemProperties().isEmpty()));
+    invokeInEveryVM(() -> assertNotNull(getSystem()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4VMDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4VMDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4VMDUnitTest.java
new file mode 100644
index 0000000..3861234
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4VMDUnitTest.java
@@ -0,0 +1,201 @@
+/*
+ * 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.test.dunit.tests;
+
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+
+import java.io.Serializable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.RMIException;
+import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+/**
+ * This class tests the functionality of the {@link VM} class.
+ */
+@Category(DistributedTest.class)
+public class JUnit4VMDUnitTest extends JUnit4DistributedTestCase {
+
+  private static final AtomicInteger COUNTER = new AtomicInteger();
+  private static final boolean BOOLEAN_VALUE = true;
+  private static final byte BYTE_VALUE = (byte) 40;
+  private static final long LONG_VALUE = 42L;
+  private static final String STRING_VALUE = "BLAH BLAH BLAH";
+
+  @Test
+  public void testInvokeStaticBoolean() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(BOOLEAN_VALUE, (boolean) vm.invoke(() -> remoteBooleanMethod()));
+  }
+
+  @Test
+  public void testInvokeStaticByte() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(BYTE_VALUE, (byte) vm.invoke(() -> remoteByteMethod()));
+  }
+
+  @Test
+  public void testInvokeStaticLong() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(LONG_VALUE, (long) vm.invoke(() -> remoteLongMethod()));
+  }
+
+  @Test
+  public void testInvokeInstance() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    assertEquals(STRING_VALUE, vm.invoke(new ClassWithString(), "getString"));
+  }
+
+  @Test
+  public void testInvokeRunnableWithException() {
+    Host host = Host.getHost(0);
+    VM vm = host.getVM(0);
+    try {
+      vm.invoke(new InvokeRunnable());
+      fail("Should have thrown a BasicTestException");
+    } catch (RMIException ex) {
+      assertTrue(ex.getCause() instanceof BasicTestException);
+    }
+  }
+
+  @Test
+  public void testReturnValue() throws Exception {
+    final Host host = Host.getHost(0);
+    final VM vm = host.getVM(0);
+    // Assert class static invocation works
+    AsyncInvocation a1 = vm.invokeAsync(() -> getAndIncStaticCount());
+    a1.join();
+    assertEquals(new Integer(0), a1.getReturnValue());
+    // Assert class static invocation with args works
+    a1 = vm.invokeAsync(() -> incrementStaticCount(new Integer(2)));
+    a1.join();
+    assertEquals(new Integer(3), a1.getReturnValue());
+    // Assert that previous values are not returned when invoking method w/ no return val
+    a1 = vm.invokeAsync(() -> incStaticCount());
+    a1.join();
+    assertNull(a1.getReturnValue());
+    // Assert that previous null returns are over-written
+    a1 = vm.invokeAsync(() -> getAndIncStaticCount());
+    a1.join();
+    assertEquals(new Integer(4), a1.getReturnValue());
+
+    // Assert object method invocation works with zero arg method
+    final VMTestObject o = new VMTestObject(0);
+    a1 = vm.invokeAsync(o, "incrementAndGet", new Object[] {});
+    a1.join();
+    assertEquals(new Integer(1), a1.getReturnValue());
+    // Assert object method invocation works with no return
+    a1 = vm.invokeAsync(o, "set", new Object[] {new Integer(3)});
+    a1.join();
+    assertNull(a1.getReturnValue());
+  }
+
+  private static Integer getAndIncStaticCount() {
+    return new Integer(COUNTER.getAndIncrement());
+  }
+
+  private static Integer incrementStaticCount(Integer inc) {
+    return new Integer(COUNTER.addAndGet(inc.intValue()));
+  }
+
+  private static void incStaticCount() {
+    COUNTER.incrementAndGet();
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   */
+  private static byte remoteByteMethod() {
+    return BYTE_VALUE;
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   */
+  private static boolean remoteBooleanMethod() {
+    return BOOLEAN_VALUE;
+  }
+
+  /**
+   * Accessed via reflection.  DO NOT REMOVE
+   */
+  private static long remoteLongMethod() {
+    return LONG_VALUE;
+  }
+
+  private static class ClassWithLong implements Serializable {
+    public long getLong() {
+      return LONG_VALUE;
+    }
+  }
+
+  private static class ClassWithByte implements Serializable {
+    public byte getByte() {
+      return BYTE_VALUE;
+    }
+  }
+
+  private static class InvokeRunnable implements SerializableRunnableIF {
+    public void run() {
+      throw new BasicTestException();
+    }
+  }
+
+  private static class ClassWithString implements Serializable {
+    public String getString() {
+      return STRING_VALUE;
+    }
+  }
+
+  private static class BasicTestException extends RuntimeException {
+    BasicTestException() {
+      this("Test exception.  Please ignore.");
+    }
+    BasicTestException(String s) {
+      super(s);
+    }
+  }
+
+  private static class VMTestObject implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final AtomicInteger val;
+    public VMTestObject(int init) {
+      this.val = new AtomicInteger(init);
+    }
+    public Integer get() {
+      return new Integer(this.val.get());
+    }
+    public Integer incrementAndGet() {
+      return new Integer(this.val.incrementAndGet());
+    }
+    public void set(Integer newVal) {
+      this.val.set(newVal.intValue());
+    }
+  }
+}


[32/32] incubator-geode git commit: GEODE-17: fix compilation errors after merge from develop

Posted by ji...@apache.org.
GEODE-17: fix compilation errors after merge from develop


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 151472c7095bff612bd38272391fcc040bec428f
Parents: 31c6054
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu Apr 14 20:06:22 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Apr 14 20:06:22 2016 -0700

----------------------------------------------------------------------
 .../cli/commands/CliCommandTestBase.java        | 21 ++--------
 .../cli/commands/DeployCommandsDUnitTest.java   | 19 ++++-----
 .../cli/commands/IndexCommandsDUnitTest.java    | 19 +++++----
 .../WanCommandPauseResumeDUnitTest.java         | 42 ++++++++++----------
 4 files changed, 41 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/151472c7/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
index 7d65ce7..de239d7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
@@ -25,6 +25,8 @@ import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
@@ -48,23 +50,6 @@ import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
 import org.junit.runners.Parameterized;
 
-import java.io.IOException;
-import java.io.PrintStream;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-import static com.gemstone.gemfire.test.dunit.LogWriterUtils.getLogWriter;
-
 /**
  * Base class for all the CLI/gfsh command dunit tests.
  */
@@ -283,7 +268,7 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
     CommandResult result = executeCommand(shell, command.toString());
 
     if (!shell.isConnectedAndReady()) {
-      throw new TestException(
+      throw new AssertionError(
           "Connect command failed to connect to manager " + endpoint + " result=" + commandResultToString(result));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/151472c7/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
index a314681..db84f84 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommandsDUnitTest.java
@@ -16,6 +16,14 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.Properties;
+import java.util.regex.Pattern;
+
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -36,20 +44,11 @@ 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.DistributedTest;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.util.Properties;
-import java.util.regex.Pattern;
-
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-
 /**
  * Unit tests for the DeployCommands class
  *
@@ -57,8 +56,6 @@ import static com.gemstone.gemfire.test.dunit.Assert.*;
  */
 @Category(DistributedTest.class)
 @SuppressWarnings("serial")
-@Category(DistributedTest.class)
-@SuppressWarnings("serial")
 @RunWith(Parameterized.class)
 public class DeployCommandsDUnitTest extends CliCommandTestBase {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/151472c7/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
index ab8097c..70aba5d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommandsDUnitTest.java
@@ -16,6 +16,12 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.DiskStoreFactory;
@@ -45,17 +51,10 @@ import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
-import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Properties;
-
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-
 @Category({ DistributedTest.class, FlakyTest.class }) // see GEODE-689, GEODE-1048
 @RunWith(Parameterized.class)
 public class IndexCommandsDUnitTest extends CliCommandTestBase {
@@ -638,7 +637,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
     Properties managerProps = new Properties();
     managerProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     managerProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost:" + locatorPort);
-    createDefaultSetup(managerProps);
+    setUpJmxManagerOnVm0ThenConnect(managerProps);
 
     // Create a cache in VM 1
     VM vm = Host.getHost(0).getVM(1);
@@ -756,7 +755,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
 
   private void setupSystem() {
     disconnectAllFromDS();
-    createDefaultSetup(null);
+    setUpJmxManagerOnVm0ThenConnect(null);
     final String parRegName = "StocksParReg";
 
     final VM manager = Host.getHost(0).getVM(0);
@@ -786,7 +785,7 @@ public class IndexCommandsDUnitTest extends CliCommandTestBase {
 
   private void setupSystemPersist() {
     disconnectAllFromDS();
-    createDefaultSetup(null);
+    setUpJmxManagerOnVm0ThenConnect(null);
     final String parRegName = "StocksParReg";
 
     final VM manager = Host.getHost(0).getVM(0);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/151472c7/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
index 5a352c1..17986e0 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
@@ -81,26 +81,26 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
   @Test
   public void testPauseGatewaySender_onMember() {
 
-    Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
+    Integer punePort = (Integer) vm1.invoke(() -> createFirstLocatorWithDSId( 1 ));
 
     Properties props = getDistributedSystemProperties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     props.setProperty(DistributionConfig.DISTRIBUTED_SYSTEM_ID_NAME, "1");
     props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + punePort + "]");
-    createDefaultSetup(props);
+    setUpJmxManagerOnVm0ThenConnect(props);
 
-    Integer nyPort = (Integer) vm2.invoke(() -> WANCommandTestBase.createFirstRemoteLocator( 2, punePort ));
+    Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator( 2, punePort ));
 
-    vm3.invoke(() -> WANCommandTestBase.createCache( punePort ));
-    vm3.invoke(() -> WANCommandTestBase.createSender( "ln",
+    vm3.invoke(() -> createCache( punePort ));
+    vm3.invoke(() -> createSender( "ln",
         2, false, 100, 400, false, false, null, true ));
 
-    vm3.invoke(() -> WANCommandTestBase.startSender( "ln" ));
+    vm3.invoke(() -> startSender( "ln" ));
 
-    vm3.invoke(() -> WANCommandTestBase.verifySenderState(
+    vm3.invoke(() -> verifySenderState(
         "ln", true, false ));
 
-    final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
+    final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> getMember());
     pause(10000);
     String command = CliStrings.PAUSE_GATEWAYSENDER + " --"
         + CliStrings.PAUSE_GATEWAYSENDER__ID + "=ln --"
@@ -116,7 +116,7 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
       fail("testPauseGatewaySender failed as did not get CommandResult");
     }
 
-    vm3.invoke(() -> WANCommandTestBase.verifySenderState(
+    vm3.invoke(() -> verifySenderState(
         "ln", true, true ));
   }
 
@@ -133,23 +133,23 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
 
     Integer nyPort = (Integer) vm2.invoke(() -> createFirstRemoteLocator( 2, punePort ));
 
-    vm3.invoke(() -> WANCommandTestBase.createCache( punePort ));
-    vm3.invoke(() -> WANCommandTestBase.createSender( "ln",
+    vm3.invoke(() -> createCache( punePort ));
+    vm3.invoke(() -> createSender( "ln",
             2, false, 100, 400, false, false, null, true ));
-    vm4.invoke(() -> WANCommandTestBase.createCache( punePort ));
-    vm4.invoke(() -> WANCommandTestBase.createSender( "ln",
+    vm4.invoke(() -> createCache( punePort ));
+    vm4.invoke(() -> createSender( "ln",
             2, false, 100, 400, false, false, null, true ));
-    vm5.invoke(() -> WANCommandTestBase.createCache( punePort ));
-    vm5.invoke(() -> WANCommandTestBase.createSender( "ln",
+    vm5.invoke(() -> createCache( punePort ));
+    vm5.invoke(() -> createSender( "ln",
             2, false, 100, 400, false, false, null, true ));
 
     vm3.invoke(() -> startSender( "ln" ));
 
-    vm3.invoke(() -> WANCommandTestBase.verifySenderState(
+    vm3.invoke(() -> verifySenderState(
             "ln", true, false ));
-    vm4.invoke(() -> WANCommandTestBase.verifySenderState(
+    vm4.invoke(() -> verifySenderState(
             "ln", true, false ));
-    vm5.invoke(() -> WANCommandTestBase.verifySenderState(
+    vm5.invoke(() -> verifySenderState(
             "ln", true, false ));
 
     pause(10000);
@@ -172,11 +172,11 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
       fail("testPauseGatewaySender failed as did not get CommandResult");
     }
 
-    vm3.invoke(() -> WANCommandTestBase.verifySenderState(
+    vm3.invoke(() -> verifySenderState(
             "ln", true, true ));
-    vm4.invoke(() -> WANCommandTestBase.verifySenderState(
+    vm4.invoke(() -> verifySenderState(
             "ln", true, true ));
-    vm5.invoke(() -> WANCommandTestBase.verifySenderState(
+    vm5.invoke(() -> verifySenderState(
             "ln", true, true ));
   }
 


[12/32] incubator-geode git commit: GEODE-1220: Minor refactoring of LuceneServiceImplJUnitTest and LuceneIndexRecoveryHAJUnitTest

Posted by ji...@apache.org.
GEODE-1220: Minor refactoring of LuceneServiceImplJUnitTest and LuceneIndexRecoveryHAJUnitTest

Removed sleep with awaitility until async queue size is 0
Removed unnecessary/dead code
Renamed tests to be slightly more descriptive


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 7d5f39aeebeb345600e8608d77a2d42720871907
Parents: 6c7a0d2
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue Apr 12 15:11:18 2016 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Wed Apr 13 13:39:25 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexRecoveryHAJUnitTest.java         | 20 ++++++---
 .../internal/LuceneServiceImplJUnitTest.java    | 44 ++++++++++----------
 2 files changed, 36 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d5f39ae/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java
index 405c986..9ab6e81 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneIndexRecoveryHAJUnitTest.java
@@ -19,9 +19,10 @@
 
 package com.gemstone.gemfire.cache.lucene.internal;
 
-import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.*;
 
 import java.io.IOException;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.analysis.Analyzer;
@@ -41,6 +42,7 @@ import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
@@ -54,6 +56,7 @@ import com.gemstone.gemfire.internal.cache.EvictionAttributesImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.jayway.awaitility.Awaitility;
 
 @Category(IntegrationTest.class)
 public class LuceneIndexRecoveryHAJUnitTest {
@@ -120,6 +123,8 @@ public class LuceneIndexRecoveryHAJUnitTest {
 
   @Test
   public void recoverPersistentIndex() throws Exception {
+    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX, REGION);
+
     LuceneService service = LuceneServiceProvider.get(cache);
     service.createIndex(INDEX, REGION, Type1.fields);
 
@@ -133,8 +138,7 @@ public class LuceneIndexRecoveryHAJUnitTest {
     value = new Type1("lucene world", 1, 2L, 3.0, 4.0f);
     userRegion.put("value3", value);
 
-    // TODO flush queue
-    TimeUnit.MILLISECONDS.sleep(500);
+    waitUntilQueueEmpty(aeqId);
 
     LuceneQuery<Integer, Type1> query = service.createLuceneQueryFactory().create(INDEX, REGION, "s:world");
     LuceneQueryResults<Integer, Type1> results = query.search();
@@ -153,7 +157,6 @@ public class LuceneIndexRecoveryHAJUnitTest {
     results = query.search();
     Assert.assertEquals(3, results.size());
 
-    String aeqId = LuceneServiceImpl.getUniqueIndexName(INDEX, REGION);
     PartitionedRegion chunkRegion = (PartitionedRegion) cache.getRegion(aeqId + ".chunks");
     assertNotNull(chunkRegion);
     chunkRegion.destroyRegion();
@@ -185,8 +188,7 @@ public class LuceneIndexRecoveryHAJUnitTest {
     value = new Type1("lucene world", 1, 2L, 3.0, 4.0f);
     userRegion.put("value3", value);
 
-    // TODO flush queue
-    TimeUnit.MILLISECONDS.sleep(500);
+    waitUntilQueueEmpty(aeqId);
 
     PartitionedRegion fileRegion = (PartitionedRegion) cache.getRegion(aeqId + ".files");
     assertNotNull(fileRegion);
@@ -198,4 +200,10 @@ public class LuceneIndexRecoveryHAJUnitTest {
     LuceneQueryResults<Integer, Type1> results = query.search();
     Assert.assertEquals(3, results.size());
   }
+
+  private void waitUntilQueueEmpty(final String aeqId) {
+    // TODO flush queue
+    AsyncEventQueue queue = cache.getAsyncEventQueue(aeqId);
+    Awaitility.waitAtMost(1000, TimeUnit.MILLISECONDS).until(() -> assertEquals(0, queue.size()));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d5f39ae/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
index 3a6f38c..d0e9865 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplJUnitTest.java
@@ -39,8 +39,10 @@ import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -48,6 +50,7 @@ import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
@@ -67,23 +70,22 @@ public class LuceneServiceImplJUnitTest {
   private IndexWriter writer;
   LuceneServiceImpl service = null;
   private static final Logger logger = LogService.getLogger();
-  
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
   // lucene service will register query execution function on initialization
   @Test
   public void shouldRegisterQueryFunction() {
     Function function = FunctionService.getFunction(LuceneFunction.ID);
     assertNull(function);
 
-    cache = createBasicCache();
+    cache = getCache();
     new LuceneServiceImpl().init(cache);
 
     function = FunctionService.getFunction(LuceneFunction.ID);
     assertNotNull(function);
   }
-  
-  private GemFireCacheImpl createBasicCache() {
-    return (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").create();
-  }
 
   @After
   public void destroyCache() {
@@ -93,34 +95,35 @@ public class LuceneServiceImplJUnitTest {
     }
   }
   
-  private void getCache() {
+  private Cache getCache() {
     try {
        cache = CacheFactory.getAnyInstance();
     } catch (Exception e) {
       //ignore
     }
     if (null == cache) {
-      cache = createBasicCache();
+      cache = new CacheFactory().set("mcast-port", "0").create();
     }
+    return cache;
   }
   
-  private void getService() {
+  private LuceneService getService() {
     if (cache == null) {
       getCache();
     }
     if (service == null) {
       service = (LuceneServiceImpl)LuceneServiceProvider.get(cache);
     }
+    return service;
   }
   
   private LocalRegion createPR(String regionName, boolean isSubRegion) {
     if (isSubRegion) {
-      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
+      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION).create("root");
       LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
           createSubregion(root, regionName);
       return region;
     } else {
-      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
       LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
           create(regionName);
       return region;
@@ -134,25 +137,21 @@ public class LuceneServiceImplJUnitTest {
           createSubregion(root, regionName);
       return region;
     } else {
-      LocalRegion root = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
       LocalRegion region = (LocalRegion)cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT).
           create(regionName);
       return region;
     }
   }
-  
-  /**Test that we don't allow the user
-   * to create the region first.
-   */
+
   @Test(expected = IllegalStateException.class)
-  public void createRegionFirst() throws IOException, ParseException {
+  public void cannotCreateLuceneIndexAfterRegionHasBeenCreated() throws IOException, ParseException {
     getService();
     LocalRegion userRegion = createPR("PR1", false);
     service.createIndex("index1", "PR1", "field1", "field2", "field3");
   }
 
   @Test
-  public void testCreateIndexForPR() throws IOException, ParseException {
+  public void canCreateLuceneIndexForPR() throws IOException, ParseException {
     getService();
     service.createIndex("index1", "PR1", "field1", "field2", "field3");
     LocalRegion userRegion = createPR("PR1", false);
@@ -184,7 +183,7 @@ public class LuceneServiceImplJUnitTest {
   }
 
   @Test
-  public void testCreateIndexForPRWithAnalyzer() throws IOException, ParseException {
+  public void canCreateLuceneIndexForPRWithAnalyzer() throws IOException, ParseException {
     getService();
     StandardAnalyzer sa = new StandardAnalyzer();
     KeywordAnalyzer ka = new KeywordAnalyzer();
@@ -217,12 +216,13 @@ public class LuceneServiceImplJUnitTest {
     assertTrue(chunkPR != null);
   }
   
-  @Test (expected=UnsupportedOperationException.class)
-  public void testCreateIndexForRR() throws IOException, ParseException {
+  @Test
+  public void cannotCreateLuceneIndexForReplicateRegion() throws IOException, ParseException {
+    expectedException.expect(UnsupportedOperationException.class);
+    expectedException.expectMessage("Lucene indexes on replicated regions are not supported");
     getService();
     service.createIndex("index1", "RR1", "field1", "field2", "field3");
     createRR("RR1", false);
-    fail("Expect UnsupportedOperationException");
   }
 
 }


[17/32] incubator-geode git commit: GEODE-679 Explore removing SocketIOWithTimeout and other classes related to FD soft leak

Posted by ji...@apache.org.
GEODE-679 Explore removing SocketIOWithTimeout and other classes related to FD soft leak

SocketUtils was added to avoid a file descriptor "leak" caused by the use of NIO socket
channel selectors.  This was spurred by a HADOOP JIRA ticket that claimed that
sun.misc.Cleaners were being used to close selectors (see
https://issues.apache.org/jira/browse/HADOOP-4346).  I have verified that Cleaners are
no longer used to close selectors and that SocketUtils is not making any difference in
the number of file descriptors created in servers using NIO selectors using the (recently
deleted) FDDUnitTest with modifications to force clients to close their connections to
the server.

So, this change-set removes SocketUtils and associated classes, reverting all modifications
made to introduce it in GemFire 8.0 to use direct method invokations on sockets.


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 7b3c8cb42abc19e62829aa8662a37415afc50b75
Parents: 5c89fab
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu Apr 14 09:31:13 2016 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Thu Apr 14 09:34:07 2016 -0700

----------------------------------------------------------------------
 .../client/internal/CloseConnectionOp.java      |   0
 .../cache/client/internal/ConnectionImpl.java   |   8 +-
 .../gemfire/internal/SocketCreator.java         |  17 +-
 .../gemfire/internal/SocketIOWithTimeout.java   | 491 -------------------
 .../gemfire/internal/SocketInputStream.java     | 181 -------
 .../gemfire/internal/SocketInputWrapper.java    |  93 ----
 .../gemfire/internal/SocketOutputStream.java    | 174 -------
 .../gemstone/gemfire/internal/SocketUtils.java  | 220 ---------
 .../internal/cache/GemFireCacheImpl.java        |  14 +-
 .../cache/tier/sockets/AcceptorImpl.java        |   3 +-
 .../cache/tier/sockets/CacheClientNotifier.java | 105 +---
 .../cache/tier/sockets/CacheClientUpdater.java  |   5 +-
 .../internal/cache/tier/sockets/HandShake.java  |  13 +-
 .../internal/cache/tier/sockets/Message.java    |   3 +-
 .../cache/tier/sockets/ServerConnection.java    |   5 +-
 .../tier/sockets/ServerHandShakeProcessor.java  |  26 +-
 .../tier/sockets/command/CloseConnection.java   |   0
 .../gemfire/internal/shared/NativeCalls.java    |   2 +-
 .../gemfire/internal/tcp/Connection.java        |   5 +-
 19 files changed, 52 insertions(+), 1313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/CloseConnectionOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/CloseConnectionOp.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/CloseConnectionOp.java
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java
old mode 100644
new mode 100755
index 5016d67..c20b318
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ConnectionImpl.java
@@ -35,14 +35,10 @@ import com.gemstone.gemfire.cache.client.internal.ExecuteFunctionOp.ExecuteFunct
 import com.gemstone.gemfire.cache.client.internal.ExecuteRegionFunctionOp.ExecuteRegionFunctionOpImpl;
 import com.gemstone.gemfire.cache.client.internal.ExecuteRegionFunctionSingleHopOp.ExecuteRegionFunctionSingleHopOpImpl;
 import com.gemstone.gemfire.cache.wan.GatewaySender;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.tier.Acceptor;
 import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerQueueStatus;
@@ -109,8 +105,8 @@ public class ConnectionImpl implements Connection {
     verifySocketBufferSize(socketBufferSize, theSocket.getSendBufferSize(), "send");
     
     theSocket.setSoTimeout(handShakeTimeout);
-    out = SocketUtils.getOutputStream(theSocket);//theSocket.getOutputStream();
-    in = SocketUtils.getInputStream(theSocket);//theSocket.getInputStream();
+    out = theSocket.getOutputStream();
+    in = theSocket.getInputStream();
     this.status = handShake.greet(this, location, communicationMode);
     commBuffer = ServerConnection.allocateCommBuffer(socketBufferSize, theSocket);
     if (sender != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
index acdfbc7..9b7e5d7 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
@@ -19,8 +19,6 @@ package com.gemstone.gemfire.internal;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
 import java.net.BindException;
 import java.net.Inet4Address;
 import java.net.Inet6Address;
@@ -995,13 +993,7 @@ public class SocketCreator {
           if (optionalWatcher != null) {
             optionalWatcher.beforeConnect(socket);
           }
-          if (timeout > 0) {
-            SocketUtils.connect(socket, sockaddr, timeout);
-          }
-          else {
-            SocketUtils.connect(socket, sockaddr, 0);
-
-          }
+          socket.connect(sockaddr, Math.max(timeout,0));
           configureClientSSLSocket( socket );
           return socket;
         } 
@@ -1024,12 +1016,7 @@ public class SocketCreator {
             if (optionalWatcher != null) {
               optionalWatcher.beforeConnect(socket);
             }
-          if (timeout > 0) {
-            SocketUtils.connect(socket, sockaddr, timeout);
-            }
-            else {
-              SocketUtils.connect(socket, sockaddr, 0);
-            }
+            socket.connect(sockaddr, Math.max(timeout,0));
           }
           return socket;
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketIOWithTimeout.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketIOWithTimeout.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketIOWithTimeout.java
deleted file mode 100644
index 9fbead8..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketIOWithTimeout.java
+++ /dev/null
@@ -1,491 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- * Pivotal Additions:
- * Using a ConcurrentHashMap with a LinkedBlockingDeque instead
- * Added a cleanup thread
- * Modifications to trimIdleSelector
- * 
- */
-
-package com.gemstone.gemfire.internal;
-
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.net.SocketAddress;
-import java.net.SocketTimeoutException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectableChannel;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.SocketChannel;
-import java.nio.channels.spi.SelectorProvider;
-import java.util.Iterator;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.internal.logging.LogService;
-
-/**
- * This supports input and output streams for a socket channels. 
- * These streams can have a timeout.
- */
-public abstract class SocketIOWithTimeout {
-  
-  private static final Logger logger = LogService.getLogger();
-  
-  private SelectableChannel channel;
-  private long timeout;
-  private boolean closed = false;
-  
-  /*Pivotal Change to final*/
-  private static final SelectorPool selector = new SelectorPool();
-  
-  /*Pivotal Addition*/
-  //in seconds, the cleanup thread will first mark at the SELECTOR_TIME_OUT interval and
-  //close any selectors that have been marked on the next pass through
-  //This means that it will take approximately twice as long as SELECTOR_TIME_OUT to actually close
-  //an unused selector
-  private static final long SELECTOR_TIME_OUT = Long.getLong("gemfire.SELECTOR_TIME_OUT", 120L); 
-  
-  private static ScheduledExecutorService cleanUpExecutor = startSelectorCleanUpThread();
-
-  /*End Pivotal Addition*/
-  
-  /* A timeout value of 0 implies wait for ever. 
-   * We should have a value of timeout that implies zero wait.. i.e. 
-   * read or write returns immediately.
-   * 
-   * This will set channel to non-blocking.
-   */
-  SocketIOWithTimeout(SelectableChannel channel, long timeout) 
-                                                 throws IOException {
-    checkChannelValidity(channel);
-    
-    this.channel = channel;
-    this.timeout = timeout;
-    // Set non-blocking
-    channel.configureBlocking(false);
-  }
-  
-  void close() {
-    closed = true;
-  }
-
-  boolean isOpen() {
-    return !closed && channel.isOpen();
-  }
-
-  SelectableChannel getChannel() {
-    return channel;
-  }
-  
-  /** 
-   * Utility function to check if channel is ok.
-   * Mainly to throw IOException instead of runtime exception
-   * in case of mismatch. This mismatch can occur for many runtime
-   * reasons.
-   */
-  static void checkChannelValidity(Object channel) throws IOException {
-    if (channel == null) {
-      /* Most common reason is that original socket does not have a channel.
-       * So making this an IOException rather than a RuntimeException.
-       */
-      throw new IOException("Channel is null. Check " +
-                            "how the channel or socket is created.");
-    }
-    
-    if (!(channel instanceof SelectableChannel)) {
-      throw new IOException("Channel should be a SelectableChannel");
-    }    
-  }
-  
-  /**
-   * Performs actual IO operations. This is not expected to block.
-   *  
-   * @param buf
-   * @return number of bytes (or some equivalent). 0 implies underlying
-   *         channel is drained completely. We will wait if more IO is 
-   *         required.
-   * @throws IOException
-   */
-  abstract int performIO(ByteBuffer buf) throws IOException;  
-  
-  /**
-   * Performs one IO and returns number of bytes read or written.
-   * It waits up to the specified timeout. If the channel is 
-   * not read before the timeout, SocketTimeoutException is thrown.
-   * 
-   * @param buf buffer for IO
-   * @param ops Selection Ops used for waiting. Suggested values: 
-   *        SelectionKey.OP_READ while reading and SelectionKey.OP_WRITE while
-   *        writing. 
-   *        
-   * @return number of bytes read or written. negative implies end of stream.
-   * @throws IOException
-   */
-  int doIO(ByteBuffer buf, int ops) throws IOException {
-    
-    /* For now only one thread is allowed. If user want to read or write
-     * from multiple threads, multiple streams could be created. In that
-     * case multiple threads work as well as underlying channel supports it.
-     */
-    if (!buf.hasRemaining()) {
-      throw new IllegalArgumentException("Buffer has no data left.");
-      //or should we just return 0?
-    }
-
-    while (buf.hasRemaining()) {
-      if (closed) {
-        return -1;
-      }
-
-      try {
-        int n = performIO(buf);
-        if (n != 0) {
-          // successful io or an error.
-          return n;
-        }
-      } catch (IOException e) {
-        if (!channel.isOpen()) {
-          closed = true;
-        }
-        throw e;
-      }
-
-      //now wait for socket to be ready.
-      int count = 0;
-      try {
-        count = selector.select(channel, ops, timeout);  
-      } catch (IOException e) { //unexpected IOException.
-        closed = true;
-        throw e;
-      } 
-
-      if (count == 0) {
-        throw new SocketTimeoutException(timeoutExceptionString(channel,
-                                                                timeout, ops));
-      }
-      // otherwise the socket should be ready for io.
-    }
-    
-    return 0; // does not reach here.
-  }
-  
-  /**
-   * The contract is similar to {@link SocketChannel#connect(SocketAddress)} 
-   * with a timeout.
-   * 
-   * @see SocketChannel#connect(SocketAddress)
-   * 
-   * @param channel - this should be a {@link SelectableChannel}
-   * @param endpoint
-   * @throws IOException
-   */
-  static void connect(SocketChannel channel, 
-                      SocketAddress endpoint, int timeout) throws IOException {
-    
-    boolean blockingOn = channel.isBlocking();
-    if (blockingOn) {
-      channel.configureBlocking(false);
-    }
-    
-    try { 
-      if (channel.connect(endpoint)) {
-        return;
-      }
-
-      long timeoutLeft = timeout;
-      long endTime = (timeout > 0) ? (System.currentTimeMillis() + timeout): 0;
-      
-      while (true) {
-        // we might have to call finishConnect() more than once
-        // for some channels (with user level protocols)
-        
-        int ret = selector.select((SelectableChannel)channel, 
-                                  SelectionKey.OP_CONNECT, timeoutLeft);
-        
-        if (ret > 0 && channel.finishConnect()) {
-          return;
-        }
-        
-        if (ret == 0 ||
-            (timeout > 0 &&  
-              (timeoutLeft = (endTime - System.currentTimeMillis())) <= 0)) {
-          throw new SocketTimeoutException(
-                    timeoutExceptionString(channel, timeout, 
-                                           SelectionKey.OP_CONNECT));
-        }
-      }
-    } catch (IOException e) {
-      // javadoc for SocketChannel.connect() says channel should be closed.
-      try {
-        channel.close();
-      } catch (IOException ignored) {}
-      throw e;
-    } finally {
-      if (blockingOn && channel.isOpen()) {
-        channel.configureBlocking(true);
-      }
-    }
-  }
-  
-  /**
-   * This is similar to doIO(ByteBuffer, int)} except that it
-   * does not perform any I/O. It just waits for the channel to be ready
-   * for I/O as specified in ops.
-   * 
-   * @param ops Selection Ops used for waiting
-   * 
-   * @throws SocketTimeoutException 
-   *         if select on the channel times out.
-   * @throws IOException
-   *         if any other I/O error occurs. 
-   */
-  void waitForIO(int ops) throws IOException {
-    
-    if (selector.select(channel, ops, timeout) == 0) {
-      throw new SocketTimeoutException(timeoutExceptionString(channel, timeout,
-                                                              ops)); 
-    }
-  }
-
-  public void setTimeout(long timeoutMs) {
-    this.timeout = timeoutMs;
-  }
-    
-  private static String timeoutExceptionString(SelectableChannel channel,
-                                               long timeout, int ops) {
-    
-    String waitingFor;
-    switch(ops) {
-    
-    case SelectionKey.OP_READ :
-      waitingFor = "read"; break;
-      
-    case SelectionKey.OP_WRITE :
-      waitingFor = "write"; break;      
-      
-    case SelectionKey.OP_CONNECT :
-      waitingFor = "connect"; break;
-      
-    default :
-      waitingFor = "" + ops;  
-    }
-    
-    return timeout + " millis timeout while " +
-           "waiting for channel to be ready for " + 
-           waitingFor + ". ch : " + channel;    
-  }
-  
-  public static ScheduledExecutorService startSelectorCleanUpThread() {
-    ScheduledExecutorService cleanUpExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() {
-      public Thread newThread(final Runnable r) {
-        Thread result = new Thread(r, "selector-pool-cleanup");
-        result.setDaemon(true);
-        return result;
-      }
-    });
-    cleanUpExecutor.scheduleAtFixedRate(new Runnable(){
-        public void run() {
-          selector.trimIdleSelectors();
-        }
-      }, SELECTOR_TIME_OUT, SELECTOR_TIME_OUT, TimeUnit.SECONDS);
-    return cleanUpExecutor;
-  }
-  
-  public static void stopSelectorCleanUpThread() {
-    if (cleanUpExecutor != null) {
-      cleanUpExecutor.shutdownNow();
-    }
-  }
-  /**
-   * This maintains a pool of selectors. These selectors are closed
-   * once they are idle (unused) for a few seconds.
-   */
-  private static class SelectorPool {
-    
-    private static class SelectorInfo {
-      Selector              selector;
-      /**Pivotal Addition**/
-      LinkedBlockingDeque<SelectorInfo> deque;
-      volatile boolean markForClean = false;
-      /**End Pivotal Addition**/
- 
-      void close() {
-        if (selector != null) {
-          try {
-            selector.close();
-          } catch (IOException e) {
-            logger.warn("Unexpected exception while closing selector : ", e);
-          }
-        }
-      }    
-    }
-    
-    private final ConcurrentHashMap<SelectorProvider, LinkedBlockingDeque<SelectorInfo>> providerList = new ConcurrentHashMap<SelectorProvider, LinkedBlockingDeque<SelectorInfo>>();
-    
-    /**
-     * Waits on the channel with the given timeout using one of the 
-     * cached selectors. It also removes any cached selectors that are
-     * idle for a few seconds.
-     * 
-     * @param channel
-     * @param ops
-     * @param timeout
-     * @throws IOException
-     */
-    int select(SelectableChannel channel, int ops, long timeout) 
-                                                   throws IOException {
-     
-      SelectorInfo info = get(channel);
-      
-      SelectionKey key = null;
-      int ret = 0;
-      
-      try {
-        while (true) {
-          long start = (timeout == 0) ? 0 : System.currentTimeMillis();
-
-          key = channel.register(info.selector, ops);
-          ret = info.selector.select(timeout);
-          
-          if (ret != 0) {
-            return ret;
-          }
-          
-          /* Sometimes select() returns 0 much before timeout for 
-           * unknown reasons. So select again if required.
-           */
-          if (timeout > 0) {
-            timeout -= System.currentTimeMillis() - start;
-            if (timeout <= 0) {
-              return 0;
-            }
-          }
-          
-          if (Thread.currentThread().isInterrupted()) {
-            throw new InterruptedIOException("Interruped while waiting for " +
-                                             "IO on channel " + channel +
-                                             ". " + timeout + 
-                                             " millis timeout left.");
-          }
-        }
-      } finally {
-        if (key != null) {
-          key.cancel();
-        }
-        
-        //clear the canceled key.
-        try {
-          info.selector.selectNow();
-        } catch (IOException e) {
-          logger.info("Unexpected Exception while clearing selector : ", e); // TODO:WTF: why is this info level??
-          // don't put the selector back.
-          info.close();
-          return ret; 
-        }
-        
-        release(info);
-      }
-    }
-
-    /**
-     * Takes one selector from end of LRU list of free selectors.
-     * If there are no selectors awailable, it creates a new selector.
-     * 
-     * @param channel 
-     * @throws IOException
-     */
-    private SelectorInfo get(SelectableChannel channel) 
-                                                         throws IOException {
-      SelectorProvider provider = channel.provider();
-      
-      /**Pivotal Change**/
-      LinkedBlockingDeque<SelectorInfo> deque = providerList.get(provider);
-      if (deque == null) {
-        deque = new LinkedBlockingDeque<SelectorInfo>();
-        LinkedBlockingDeque<SelectorInfo> presentValue = providerList.putIfAbsent(provider, deque); 
-        if (presentValue != null && deque != presentValue) {
-          deque = presentValue;
-        }
-      } 
-      /**poll instead of check empty**/       
-      
-      SelectorInfo selInfo = deque.pollFirst(); 
-      if (selInfo != null) {
-        selInfo.markForClean = false;
-      } else {
-        Selector selector = provider.openSelector();
-        selInfo = new SelectorInfo();
-        selInfo.selector = selector;
-        selInfo.deque = deque;
-      }
-      
-      /**end Pivotal Change**/
-      return selInfo;
-    }
-    
-    /**
-     * puts selector back at the end of LRU list of free selectos.
-     * 
-     * @param info
-     */
-    private void release(SelectorInfo info) {
-      /**Pivotal Addition **/
-      info.deque.addFirst(info);
-      /**End Pivotal Addition **/
-    }
-    
-    private void trimIdleSelectors() {
-      Iterator<LinkedBlockingDeque<SocketIOWithTimeout.SelectorPool.SelectorInfo>> poolIterator = providerList.values().iterator();
-      while (poolIterator.hasNext()) {
-        LinkedBlockingDeque<SelectorInfo> selectorPool = poolIterator.next();
-        trimSelectorPool(selectorPool);
-      }
-    }
-        
-    private void trimSelectorPool(LinkedBlockingDeque<SelectorInfo> selectorPool) {
-      SelectorInfo selectorInfo = selectorPool.peekLast();
-      //iterate backwards and remove any selectors that have been marked
-      //once we hit a selector that has yet to be marked, we can then mark the remaining
-      while (selectorInfo != null && selectorInfo.markForClean) {
-        selectorInfo = selectorPool.pollLast();
-        //check the flag again just to be sure
-        if (selectorInfo.markForClean ) {
-          selectorInfo.close();
-        }
-        else {
-          selectorPool.addFirst(selectorInfo);
-        }
-        selectorInfo = selectorPool.peekLast();
-      }
-      
-      //Mark all the selectors
-      Iterator<SelectorInfo> selectorIterator = selectorPool.iterator();
-      while (selectorIterator.hasNext()) {
-        selectorIterator.next().markForClean = true;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketInputStream.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketInputStream.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketInputStream.java
deleted file mode 100644
index 430294a..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketInputStream.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- * Pivotal Additions:
- * Removed the usage of import org.apache.hadoop.classification.InterfaceAudience
- */
-package com.gemstone.gemfire.internal;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.Socket;
-import java.net.SocketTimeoutException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.SelectableChannel;
-import java.nio.channels.SelectionKey;
-
-/**
- * This implements an input stream that can have a timeout while reading.
- * This sets non-blocking flag on the socket channel.
- * So after create this object, read() on 
- * {@link Socket#getInputStream()} and write() on 
- * {@link Socket#getOutputStream()} for the associated socket will throw 
- * IllegalBlockingModeException. 
- * Please use {@link SocketOutputStream} for writing.
- */
-//@InterfaceAudience.LimitedPrivate("HDFS")
-public class SocketInputStream extends InputStream
-                               implements ReadableByteChannel {
-
-  private Reader reader;
-
-  private static class Reader extends SocketIOWithTimeout {
-    ReadableByteChannel channel;
-    
-    Reader(ReadableByteChannel channel, long timeout) throws IOException {
-      super((SelectableChannel)channel, timeout);
-      this.channel = channel;
-    }
-    
-    @Override
-    int performIO(ByteBuffer buf) throws IOException {
-      return channel.read(buf);
-    }
-  }
-  
-  /**
-   * Create a new input stream with the given timeout. If the timeout
-   * is zero, it will be treated as infinite timeout. The socket's
-   * channel will be configured to be non-blocking.
-   * 
-   * @param channel 
-   *        Channel for reading, should also be a {@link SelectableChannel}.
-   *        The channel will be configured to be non-blocking.
-   * @param timeout timeout in milliseconds. must not be negative.
-   * @throws IOException
-   */
-  public SocketInputStream(ReadableByteChannel channel, long timeout)
-                                                        throws IOException {
-    SocketIOWithTimeout.checkChannelValidity(channel);
-    reader = new Reader(channel, timeout);
-  }
-
-  /**
-   * Same as SocketInputStream(socket.getChannel(), timeout): <br><br>
-   * 
-   * Create a new input stream with the given timeout. If the timeout
-   * is zero, it will be treated as infinite timeout. The socket's
-   * channel will be configured to be non-blocking.
-   * 
-   * @see SocketInputStream#SocketInputStream(ReadableByteChannel, long)
-   *  
-   * @param socket should have a channel associated with it.
-   * @param timeout timeout timeout in milliseconds. must not be negative.
-   * @throws IOException
-   */
-  public SocketInputStream(Socket socket, long timeout) 
-                                         throws IOException {
-    this(socket.getChannel(), timeout);
-  }
-  
-  /**
-   * Same as SocketInputStream(socket.getChannel(), socket.getSoTimeout())
-   * :<br><br>
-   * 
-   * Create a new input stream with the given timeout. If the timeout
-   * is zero, it will be treated as infinite timeout. The socket's
-   * channel will be configured to be non-blocking.
-   * @see SocketInputStream#SocketInputStream(ReadableByteChannel, long)
-   *  
-   * @param socket should have a channel associated with it.
-   * @throws IOException
-   */
-  public SocketInputStream(Socket socket) throws IOException {
-    this(socket.getChannel(), socket.getSoTimeout());
-  }
-  
-  @Override
-  public int read() throws IOException {
-    /* Allocation can be removed if required.
-     * probably no need to optimize or encourage single byte read.
-     */
-    byte[] buf = new byte[1];
-    int ret = read(buf, 0, 1);
-    if (ret > 0) {
-      return (int)(buf[0] & 0xff);
-    }
-    if (ret != -1) {
-      // unexpected
-      throw new IOException("Could not read from stream");
-    }
-    return ret;
-  }
-
-  @Override
-  public int read(byte[] b, int off, int len) throws IOException {
-    return read(ByteBuffer.wrap(b, off, len));
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    /* close the channel since Socket.getInputStream().close()
-     * closes the socket.
-     */
-    reader.channel.close();
-    reader.close();
-  }
-
-  /**
-   * Returns underlying channel used by inputstream.
-   * This is useful in certain cases like channel for 
-   * {@link FileChannel#transferFrom(ReadableByteChannel, long, long)}.
-   */
-  public ReadableByteChannel getChannel() {
-    return reader.channel; 
-  }
-  
-  //ReadableByteChannel interface
-    
-  @Override
-  public boolean isOpen() {
-    return reader.isOpen();
-  }
-    
-  @Override
-  public int read(ByteBuffer dst) throws IOException {
-    return reader.doIO(dst, SelectionKey.OP_READ);
-  }
-  
-  /**
-   * waits for the underlying channel to be ready for reading.
-   * The timeout specified for this stream applies to this wait.
-   * 
-   * @throws SocketTimeoutException 
-   *         if select on the channel times out.
-   * @throws IOException
-   *         if any other I/O error occurs. 
-   */
-  public void waitForReadable() throws IOException {
-    reader.waitForIO(SelectionKey.OP_READ);
-  }
-
-  public void setTimeout(long timeoutMs) {
-    reader.setTimeout(timeoutMs);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketInputWrapper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketInputWrapper.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketInputWrapper.java
deleted file mode 100644
index 0bc0ecd..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketInputWrapper.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- * 
- * Pivotal Additions:
- * Removed Preconditions and classifications
- */
-package com.gemstone.gemfire.internal;
-
-import java.io.FilterInputStream;
-
-import java.io.InputStream;
-import java.net.Socket;
-import java.net.SocketException;
-import java.nio.channels.ReadableByteChannel;
-
-//import org.apache.hadoop.classification.InterfaceAudience;
-//import org.apache.hadoop.classification.InterfaceStability;
-
-//import com.google.common.base.Preconditions;
-
-/**
- * A wrapper stream around a socket which allows setting of its timeout. If the
- * socket has a channel, this uses non-blocking IO via the package-private
- * {@link SocketInputStream} implementation. Otherwise, timeouts are managed by
- * setting the underlying socket timeout itself.
- */
-/*
-@InterfaceAudience.LimitedPrivate("HDFS")
-@InterfaceStability.Unstable
-*/
-public class SocketInputWrapper extends FilterInputStream {
-  private final Socket socket;
-  private final boolean hasChannel;
-
-  SocketInputWrapper(Socket s, InputStream is) {
-    super(is);
-    this.socket = s;
-    this.hasChannel = s.getChannel() != null;
-//    if (hasChannel) {
-//      Preconditions.checkArgument(is instanceof SocketInputStream,
-//          "Expected a SocketInputStream when there is a channel. " +
-//          "Got: %s", is);
-//    }
-  }
-
-  /**
-   * Set the timeout for reads from this stream.
-   * 
-   * Note: the behavior here can differ subtly depending on whether the
-   * underlying socket has an associated Channel. In particular, if there is no
-   * channel, then this call will affect the socket timeout for <em>all</em>
-   * readers of this socket. If there is a channel, then this call will affect
-   * the timeout only for <em>this</em> stream. As such, it is recommended to
-   * only create one {@link SocketInputWrapper} instance per socket.
-   * 
-   * @param timeoutMs
-   *          the new timeout, 0 for no timeout
-   * @throws SocketException
-   *           if the timeout cannot be set
-   */
-  public void setTimeout(long timeoutMs) throws SocketException {
-    if (hasChannel) {
-      ((SocketInputStream)in).setTimeout(timeoutMs);
-    } else {
-      socket.setSoTimeout((int)timeoutMs);
-    }
-  }
-
-  /**
-   * @return an underlying ReadableByteChannel implementation.
-   * @throws IllegalStateException if this socket does not have a channel
-   */
-  public ReadableByteChannel getReadableByteChannel() {
-//    Preconditions.checkState(hasChannel,
-//        "Socket %s does not have a channel",
-//        this.socket);
-    return (SocketInputStream)in;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketOutputStream.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketOutputStream.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketOutputStream.java
deleted file mode 100644
index 412635c..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketOutputStream.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- * 
- * Pivotal Additions:
- * Removed classifications
- * Removed method transferToFully
- * 
- */
-package com.gemstone.gemfire.internal;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.Socket;
-import java.net.SocketTimeoutException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SelectableChannel;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.WritableByteChannel;
-
-/**
- * This implements an output stream that can have a timeout while writing.
- * This sets non-blocking flag on the socket channel.
- * So after creating this object , read() on 
- * {@link Socket#getInputStream()} and write() on 
- * {@link Socket#getOutputStream()} on the associated socket will throw 
- * llegalBlockingModeException.
- * Please use {@link SocketInputStream} for reading.
- */
-public class SocketOutputStream extends OutputStream 
-                                implements WritableByteChannel {                                
-  
-  private Writer writer;
-  
-  private static class Writer extends SocketIOWithTimeout {
-    WritableByteChannel channel;
-    
-    Writer(WritableByteChannel channel, long timeout) throws IOException {
-      super((SelectableChannel)channel, timeout);
-      this.channel = channel;
-    }
-    
-    @Override
-    int performIO(ByteBuffer buf) throws IOException {
-      return channel.write(buf);
-    }
-  }
-  
-  /**
-   * Create a new ouput stream with the given timeout. If the timeout
-   * is zero, it will be treated as infinite timeout. The socket's
-   * channel will be configured to be non-blocking.
-   * 
-   * @param channel 
-   *        Channel for writing, should also be a {@link SelectableChannel}.  
-   *        The channel will be configured to be non-blocking.
-   * @param timeout timeout in milliseconds. must not be negative.
-   * @throws IOException
-   */
-  public SocketOutputStream(WritableByteChannel channel, long timeout) 
-                                                         throws IOException {
-    SocketIOWithTimeout.checkChannelValidity(channel);
-    writer = new Writer(channel, timeout);
-  }
-  
-  /**
-   * Same as SocketOutputStream(socket.getChannel(), timeout):<br><br>
-   * 
-   * Create a new ouput stream with the given timeout. If the timeout
-   * is zero, it will be treated as infinite timeout. The socket's
-   * channel will be configured to be non-blocking.
-   * 
-   * @see SocketOutputStream#SocketOutputStream(WritableByteChannel, long)
-   *  
-   * @param socket should have a channel associated with it.
-   * @param timeout timeout timeout in milliseconds. must not be negative.
-   * @throws IOException
-   */
-  public SocketOutputStream(Socket socket, long timeout) 
-                                         throws IOException {
-    this(socket.getChannel(), timeout);
-  }
-  
-  @Override
-  public void write(int b) throws IOException {
-    /* If we need to, we can optimize this allocation.
-     * probably no need to optimize or encourage single byte writes.
-     */
-    byte[] buf = new byte[1];
-    buf[0] = (byte)b;
-    write(buf, 0, 1);
-  }
-  
-  @Override
-  public void write(byte[] b, int off, int len) throws IOException {
-    ByteBuffer buf = ByteBuffer.wrap(b, off, len);
-    while (buf.hasRemaining()) {
-      try {
-        if (write(buf) < 0) {
-          throw new IOException("The stream is closed");
-        }
-      } catch (IOException e) {
-        /* Unlike read, write can not inform user of partial writes.
-         * So will close this if there was a partial write.
-         */
-        if (buf.capacity() > buf.remaining()) {
-          writer.close();
-        }
-        throw e;
-      }
-    }
-  }
-
-  @Override
-  public synchronized void close() throws IOException {
-    /* close the channel since Socket.getOuputStream().close() 
-     * closes the socket.
-     */
-    writer.channel.close();
-    writer.close();
-  }
-
-  /**
-   * Returns underlying channel used by this stream.
-   * This is useful in certain cases like channel for 
-   * {@link FileChannel#transferTo(long, long, WritableByteChannel)}
-   */
-  public WritableByteChannel getChannel() {
-    return writer.channel; 
-  }
-
-  //WritableByteChannle interface 
-  
-  @Override
-  public boolean isOpen() {
-    return writer.isOpen();
-  }
-
-  @Override
-  public int write(ByteBuffer src) throws IOException {
-    return writer.doIO(src, SelectionKey.OP_WRITE);
-  }
-  
-  /**
-   * waits for the underlying channel to be ready for writing.
-   * The timeout specified for this stream applies to this wait.
-   *
-   * @throws SocketTimeoutException 
-   *         if select on the channel times out.
-   * @throws IOException
-   *         if any other I/O error occurs. 
-   */
-  public void waitForWritable() throws IOException {
-    writer.waitForIO(SelectionKey.OP_WRITE);
-  }
-
-  public void setTimeout(int timeoutMs) {
-    writer.setTimeout(timeoutMs);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketUtils.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketUtils.java
deleted file mode 100644
index 36eaf04..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketUtils.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- * 
- * Pivotal Additions:
- * Flag to enable/disable selector pooling for test purposes
- * 
- */
-package com.gemstone.gemfire.internal;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.ConnectException;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.net.SocketTimeoutException;
-import java.nio.channels.SocketChannel;
-
-public class SocketUtils {
-
-  //used for testing
-  public static boolean USE_SELECTOR_POOLING = Boolean.valueOf(System.getProperty("gemfire.useSelectorPooling", "true")).booleanValue();
-  /**
-   * This is a drop-in replacement for 
-   * {@link Socket#connect(SocketAddress, int)}.
-   * In the case of normal sockets that don't have associated channels, this 
-   * just invokes <code>socket.connect(endpoint, timeout)</code>. If 
-   * <code>socket.getChannel()</code> returns a non-null channel,
-   * connect is implemented using Hadoop's selectors. This is done mainly
-   * to avoid Sun's connect implementation from creating thread-local 
-   * selectors, since Hadoop does not have control on when these are closed
-   * and could end up taking all the available file descriptors.
-   * 
-   * @see java.net.Socket#connect(java.net.SocketAddress, int)
-   * 
-   * @param socket
-   * @param address the remote address
-   * @param timeout timeout in milliseconds
-   */
-  public static void connect(Socket socket,
-      SocketAddress address,
-      int timeout) throws IOException {
-    connect(socket, address, null, timeout);
-  }
-  
-  /**
-   * Like SocketUtils.connect(Socket, SocketAddress, int) but
-   * also takes a local address and port to bind the socket to. 
-   * 
-   * @param socket
-   * @param endpoint the remote address
-   * @param localAddr the local address to bind the socket to
-   * @param timeout timeout in milliseconds
-   */
-  public static void connect(Socket socket, 
-                             SocketAddress endpoint,
-                             SocketAddress localAddr,
-                             int timeout) throws IOException {
-    if (socket == null || endpoint == null || timeout < 0) {
-      throw new IllegalArgumentException("Illegal argument for connect()");
-    }
-    SocketChannel ch = socket.getChannel();
-    
-    if (localAddr != null) {
-      socket.bind(localAddr);
-    }
-
-    try {
-      if (ch == null) {
-        // let the default implementation handle it.
-        socket.connect(endpoint, timeout);
-      } else {
-        if (USE_SELECTOR_POOLING) {
-          SocketIOWithTimeout.connect(ch, endpoint, timeout);
-        }
-        else {
-          socket.connect(endpoint, timeout);
-        }
-
-      }
-    } catch (SocketTimeoutException ste) {
-      throw new IOException(ste.getMessage());
-    }
-
-    /*
-     Pivotal Change: due to ticket #50734
-    // There is a very rare case allowed by the TCP specification, such that
-    // if we are trying to connect to an endpoint on the local machine,
-    // and we end up choosing an ephemeral port equal to the destination port,
-    // we will actually end up getting connected to ourself (ie any data we
-    // send just comes right back). This is only possible if the target
-    // daemon is down, so we'll treat it like connection refused.
-    if (socket.getLocalPort() == socket.getPort() &&
-        socket.getLocalAddress().equals(socket.getInetAddress())) {
-      socket.close();
-      throw new ConnectException(
-        "Localhost targeted connection resulted in a loopback. " +
-        "No daemon is listening on the target port.");
-    }
-    */
-  }
-  
-  /**
-   * Same as <code>getInputStream(socket, socket.getSoTimeout()).</code>
-   * <br><br>
-   * 
-   * @see #getInputStream(Socket, long)
-   */
-  public static InputStream getInputStream(Socket socket) 
-                                           throws IOException {
-    return getInputStream(socket, socket.getSoTimeout());
-  }
-
-  /**
-   * Return a {@link SocketInputWrapper} for the socket and set the given
-   * timeout. If the socket does not have an associated channel, then its socket
-   * timeout will be set to the specified value. Otherwise, a
-   * {@link SocketInputStream} will be created which reads with the configured
-   * timeout.
-   * 
-   * Any socket created using socket factories returned by {@link #SocketUtils},
-   * must use this interface instead of {@link Socket#getInputStream()}.
-   * 
-   * In general, this should be called only once on each socket: see the note
-   * in {@link SocketInputWrapper#setTimeout(long)} for more information.
-   *
-   * @see Socket#getChannel()
-   * 
-   * @param socket
-   * @param timeout timeout in milliseconds. zero for waiting as
-   *                long as necessary.
-   * @return SocketInputWrapper for reading from the socket.
-   * @throws IOException
-   */
-  /*Pivotal Addition
-   * Return type changed to InputStream instead of SocketInputWrapper
-   * Returning the regular inputstream if a channel is not present and does
-   * not wrap that around an input wrapper
-   */
-  public static InputStream getInputStream(Socket socket, long timeout) 
-                                           throws IOException {
-    if (socket.getChannel() == null || ! USE_SELECTOR_POOLING) {
-      return socket.getInputStream();
-    }
-    else {
-      SocketInputWrapper w = new SocketInputWrapper(socket, new SocketInputStream(socket));
-      w.setTimeout(timeout);
-      return w;
-    }
-  }
-  
-  /**
-   * Same as getOutputStream(socket, 0). Timeout of zero implies write will
-   * wait until data is available.<br><br>
-   * 
-   * From documentation for {@link #getOutputStream(Socket, long)} : <br>
-   * Returns OutputStream for the socket. If the socket has an associated
-   * SocketChannel then it returns a 
-   * {@link SocketOutputStream} with the given timeout. If the socket does not
-   * have a channel, {@link Socket#getOutputStream()} is returned. In the later
-   * case, the timeout argument is ignored and the write will wait until 
-   * data is available.<br><br>
-   * 
-   * Any socket created using socket factories returned by {@link SocketUtils},
-   * must use this interface instead of {@link Socket#getOutputStream()}.
-   * 
-   * @see #getOutputStream(Socket, long)
-   * 
-   * @param socket
-   * @return OutputStream for writing to the socket.
-   * @throws IOException
-   */  
-  public static OutputStream getOutputStream(Socket socket) 
-                                             throws IOException {
-    return getOutputStream(socket, 0);
-  }
-  
-  /**
-   * Returns OutputStream for the socket. If the socket has an associated
-   * SocketChannel then it returns a 
-   * {@link SocketOutputStream} with the given timeout. If the socket does not
-   * have a channel, {@link Socket#getOutputStream()} is returned. In the later
-   * case, the timeout argument is ignored and the write will wait until 
-   * data is available.<br><br>
-   * 
-   * Any socket created using socket factories returned by {@link SocketUtils},
-   * must use this interface instead of {@link Socket#getOutputStream()}.
-   * 
-   * @see Socket#getChannel()
-   * 
-   * @param socket
-   * @param timeout timeout in milliseconds. This may not always apply. zero
-   *        for waiting as long as necessary.
-   * @return OutputStream for writing to the socket.
-   * @throws IOException   
-   */
-  public static OutputStream getOutputStream(Socket socket, long timeout) 
-                                             throws IOException {
-    if (socket.getChannel() == null || !USE_SELECTOR_POOLING) {
-      return socket.getOutputStream();      
-    }
-    else {
-      return new SocketOutputStream(socket, timeout); 
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index 05bc838..cc9727b 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -176,7 +176,6 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.ClassPathLoader;
 import com.gemstone.gemfire.internal.JarDeployer;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.SocketIOWithTimeout;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.cache.control.InternalResourceManager;
 import com.gemstone.gemfire.internal.cache.control.InternalResourceManager.ResourceType;
@@ -2072,9 +2071,9 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
           // ignore
         }
 
-          GatewaySenderAdvisor advisor = null;
-          for (GatewaySender sender : this.getAllGatewaySenders()) {
-            try {
+        GatewaySenderAdvisor advisor = null;
+        for (GatewaySender sender : this.getAllGatewaySenders()) {
+          try {
             sender.stop();
             advisor = ((AbstractGatewaySender) sender).getSenderAdvisor();
             if (advisor != null) {
@@ -2083,10 +2082,10 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
               }
               advisor.close();
             }
-            } catch (CancelException ce) {
-            }
+          } catch (CancelException ce) {
           }
-          ParallelGatewaySenderQueue.cleanUpStatics(null);
+        }
+        ParallelGatewaySenderQueue.cleanUpStatics(null);
 
         destroyGatewaySenderLockService();
 
@@ -2350,7 +2349,6 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
       SequenceLoggerImpl.signalCacheClose();
       SystemFailure.signalCacheClose();
       
-      SocketIOWithTimeout.stopSelectorCleanUpThread();
     } // static synchronization on GemFireCache.class
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
index 5b20e86..eeb611e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
@@ -74,7 +74,6 @@ import com.gemstone.gemfire.distributed.internal.LonerDistributionManager;
 import com.gemstone.gemfire.distributed.internal.PooledExecutorWithDMStats;
 import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.cache.BucketAdvisor;
 import com.gemstone.gemfire.internal.cache.BucketAdvisor.BucketProfile;
@@ -1483,7 +1482,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
       }
     } else {
       s.setSoTimeout(this.acceptTimeout);
-      communicationMode = (byte)SocketUtils.getInputStream(s).read();//getInputStream().read();
+      communicationMode = (byte)s.getInputStream().read();
       if (logger.isTraceEnabled()) {
         logger.trace("read communications mode(2) ", communicationMode);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
index 3178b8d..1ba2294 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
@@ -17,101 +17,23 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.io.BufferedOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.security.Principal;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.CancelException;
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.Instantiator;
-import com.gemstone.gemfire.InternalGemFireError;
-import com.gemstone.gemfire.StatisticsFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheEvent;
-import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.InterestRegistrationEvent;
-import com.gemstone.gemfire.cache.InterestRegistrationListener;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionDestroyedException;
-import com.gemstone.gemfire.cache.RegionExistsException;
-import com.gemstone.gemfire.cache.UnsupportedVersionException;
+import com.gemstone.gemfire.*;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl.PoolTask;
 import com.gemstone.gemfire.cache.query.CqException;
 import com.gemstone.gemfire.cache.query.Query;
 import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
-import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
 import com.gemstone.gemfire.cache.query.internal.cq.ServerCQ;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DM;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.HighPriorityDistributionMessage;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.distributed.internal.MessageWithReply;
-import com.gemstone.gemfire.distributed.internal.ReplyMessage;
-import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
-import com.gemstone.gemfire.internal.ClassLoadUtil;
-import com.gemstone.gemfire.internal.DummyStatisticsFactory;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-import com.gemstone.gemfire.internal.InternalInstantiator;
-import com.gemstone.gemfire.internal.SocketCloser;
-import com.gemstone.gemfire.internal.SocketUtils;
-import com.gemstone.gemfire.internal.SystemTimer;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.VersionedDataInputStream;
-import com.gemstone.gemfire.internal.VersionedDataOutputStream;
-import com.gemstone.gemfire.internal.cache.ClientServerObserver;
-import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
-import com.gemstone.gemfire.internal.cache.ClientRegionEventImpl;
-import com.gemstone.gemfire.internal.cache.CacheServerImpl;
-import com.gemstone.gemfire.internal.cache.CacheClientStatus;
-import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor;
-import com.gemstone.gemfire.internal.cache.CachedDeserializable;
-import com.gemstone.gemfire.internal.cache.Conflatable;
-import com.gemstone.gemfire.internal.cache.DistributedRegion;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
-import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
-import com.gemstone.gemfire.internal.cache.EventID;
-import com.gemstone.gemfire.internal.cache.FilterProfile;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl.SerializedCacheValueImpl;
+import com.gemstone.gemfire.distributed.internal.*;
+import com.gemstone.gemfire.internal.*;
+import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo.FilterInfo;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.InternalCacheEvent;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.RegionEventImpl;
-import com.gemstone.gemfire.internal.cache.ha.HAContainerMap;
-import com.gemstone.gemfire.internal.cache.ha.HAContainerRegion;
-import com.gemstone.gemfire.internal.cache.ha.HAContainerWrapper;
-import com.gemstone.gemfire.internal.cache.ha.HARegionQueue;
-import com.gemstone.gemfire.internal.cache.ha.ThreadIdentifier;
+import com.gemstone.gemfire.internal.cache.ha.*;
 import com.gemstone.gemfire.internal.cache.tier.Acceptor;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
@@ -122,6 +44,15 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.security.AccessControl;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.AuthenticationRequiredException;
+import org.apache.logging.log4j.Logger;
+
+import java.io.*;
+import java.lang.reflect.Method;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.security.Principal;
+import java.util.*;
+import java.util.concurrent.*;
 
 /**
  * Class <code>CacheClientNotifier</code> works on the server and manages
@@ -314,8 +245,8 @@ public class CacheClientNotifier {
   {
     // Since no remote ports were specified in the message, wait for them.
     long startTime = this._statistics.startTime();
-    DataInputStream dis = new DataInputStream(SocketUtils.getInputStream(socket));//socket.getInputStream());
-    DataOutputStream dos = new DataOutputStream(SocketUtils.getOutputStream(socket));//socket.getOutputStream());
+    DataInputStream dis = new DataInputStream(socket.getInputStream());
+    DataOutputStream dos = new DataOutputStream(socket.getOutputStream());
 
     // Read the client version
     short clientVersionOrdinal = Version.readOrdinal(dis);
@@ -607,7 +538,7 @@ public class CacheClientNotifier {
     // is attempted to be registered or authentication fails.
     try {
       DataOutputStream dos = new DataOutputStream(new BufferedOutputStream(
-          SocketUtils.getOutputStream(socket)));//socket.getOutputStream()));
+          socket.getOutputStream()));
       // write the message type, message length and the error message (if any)
       writeMessage(dos, responseByte, unsuccessfulMsg, clientVersion, epType, qSize);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
index 3e43b69..8968f62 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
@@ -71,7 +71,6 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.InternalInstantiator;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.StatisticsTypeFactoryImpl;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.ClientServerObserver;
@@ -328,8 +327,8 @@ public class CacheClientUpdater extends Thread implements ClientUpdater,
 
       // set the timeout for the handshake
       mySock.setSoTimeout(handshakeTimeout);
-      tmpOut = SocketUtils.getOutputStream(mySock);
-      tmpIn = SocketUtils.getInputStream(mySock);
+      tmpOut = mySock.getOutputStream();
+      tmpIn = mySock.getInputStream();
 
       if (isDebugEnabled) {
         logger.debug("Initialized server-to-client socket with send buffer size: {} bytes and receive buffer size: {} bytes", mySock.getSendBufferSize(), mySock.getReceiveBufferSize());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
old mode 100644
new mode 100755
index 2ea6ca0..909b133
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
@@ -80,7 +80,6 @@ import com.gemstone.gemfire.internal.ClassLoadUtil;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.InternalInstantiator;
-import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.VersionedDataInputStream;
 import com.gemstone.gemfire.internal.VersionedDataOutputStream;
@@ -257,7 +256,7 @@ public class HandShake implements ClientHandShake
       try {
         soTimeout = sock.getSoTimeout();
         sock.setSoTimeout(timeout);
-        InputStream is = SocketUtils.getInputStream(sock);//sock.getInputStream();
+        InputStream is = sock.getInputStream();
         int valRead =  is.read();
         //this.code =  (byte)is.read();
         if (valRead == -1) {
@@ -269,7 +268,7 @@ public class HandShake implements ClientHandShake
         }
         try {
           DataInputStream dis = new DataInputStream(is);
-          DataOutputStream dos = new DataOutputStream(SocketUtils.getOutputStream(sock));//sock.getOutputStream());
+          DataOutputStream dos = new DataOutputStream(sock.getOutputStream());
           this.clientReadTimeout = dis.readInt();
           if (clientVersion.compareTo(Version.CURRENT) < 0) {
             // versioned streams allow object serialization code to deal with older clients
@@ -1272,8 +1271,8 @@ public class HandShake implements ClientHandShake
     try {
       ServerQueueStatus serverQStatus = null;
       Socket sock = conn.getSocket();
-      DataOutputStream dos = new DataOutputStream(SocketUtils.getOutputStream(sock));//sock.getOutputStream());
-      final InputStream in = SocketUtils.getInputStream(sock);//sock.getInputStream();
+      DataOutputStream dos = new DataOutputStream(sock.getOutputStream());
+      final InputStream in = sock.getInputStream();
       DataInputStream dis = new DataInputStream(in);
       DistributedMember member = getDistributedMember(sock);
       // if running in a loner system, use the new port number in the ID to 
@@ -1378,8 +1377,8 @@ public class HandShake implements ClientHandShake
       AuthenticationFailedException, ServerRefusedConnectionException, ClassNotFoundException {
     ServerQueueStatus sqs = null;
     try {
-      DataOutputStream dos = new DataOutputStream(SocketUtils.getOutputStream(sock));//sock.getOutputStream());
-      final InputStream in = SocketUtils.getInputStream(sock);//sock.getInputStream());
+      DataOutputStream dos = new DataOutputStream(sock.getOutputStream());
+      final InputStream in = sock.getInputStream();
       DataInputStream dis = new DataInputStream(in);
       DistributedMember member = getDistributedMember(sock);
       if (!this.multiuserSecureMode) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java
index bfe382c..94b4953 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/Message.java
@@ -32,7 +32,6 @@ import org.apache.logging.log4j.Logger;
 import com.gemstone.gemfire.SerializationException;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.TXManagerImpl;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
@@ -1036,7 +1035,7 @@ public class Message  {
   public void setComms(Socket socket, ByteBuffer bb, MessageStats msgStats) throws IOException {
     this.sockCh = socket.getChannel();
     if (this.sockCh == null) {
-      setComms(socket, SocketUtils.getInputStream(socket), SocketUtils.getOutputStream(socket), bb, msgStats);
+      setComms(socket, socket.getInputStream(), socket.getOutputStream(), bb, msgStats);
     } else {
       setComms(socket, null, null,  bb, msgStats);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
old mode 100644
new mode 100755
index 1dd2562..e608db3
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
@@ -27,7 +27,6 @@ import java.nio.channels.SelectableChannel;
 import java.nio.channels.SelectionKey;
 import java.nio.channels.Selector;
 import java.security.Principal;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
@@ -42,13 +41,11 @@ import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.client.internal.AbstractOp;
 import com.gemstone.gemfire.cache.client.internal.Connection;
-import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -630,7 +627,7 @@ public class ServerConnection implements Runnable {
   private boolean acceptHandShake(byte epType, int qSize)
   {
     try {
-      this.handshake.accept(SocketUtils.getOutputStream(theSocket), SocketUtils.getInputStream(this.theSocket)//this.theSocket
+      this.handshake.accept(theSocket.getOutputStream(), theSocket.getInputStream()
           , epType, qSize, this.communicationMode,
           this.principal);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerHandShakeProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerHandShakeProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerHandShakeProcessor.java
index 07185b8..867f397 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerHandShakeProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerHandShakeProcessor.java
@@ -17,20 +17,6 @@
 
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.net.SocketException;
-import java.net.SocketTimeoutException;
-import java.security.Principal;
-import java.util.Properties;
-
-import org.apache.logging.log4j.Logger;
-
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.IncompatibleVersionException;
 import com.gemstone.gemfire.cache.UnsupportedVersionException;
@@ -40,7 +26,6 @@ import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
-import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.VersionedDataStream;
 import com.gemstone.gemfire.internal.cache.tier.Acceptor;
@@ -52,6 +37,15 @@ import com.gemstone.gemfire.internal.security.AuthorizeRequest;
 import com.gemstone.gemfire.internal.security.AuthorizeRequestPP;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.AuthenticationRequiredException;
+import org.apache.logging.log4j.Logger;
+
+import java.io.*;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.security.Principal;
+import java.util.Properties;
 
 /**
  * A <code>ServerHandShakeProcessor</code> verifies the client's version compatibility with server.
@@ -422,7 +416,7 @@ public class ServerHandShakeProcessor {
     try {
       soTimeout = socket.getSoTimeout();
       socket.setSoTimeout(timeout);
-      InputStream is = SocketUtils.getInputStream(socket);//socket.getInputStream();
+      InputStream is = socket.getInputStream();
       short clientVersionOrdinal = Version.readOrdinalFromInputStream(is);
       if (clientVersionOrdinal == -1) {
         throw new EOFException(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseConnection.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseConnection.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseConnection.java
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java
old mode 100644
new mode 100755
index d9dd826..f0fc27a
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java
@@ -169,7 +169,7 @@ public abstract class NativeCalls {
       throw new UnsupportedOperationException(ex);
     }
 
-    // first try using SocketInputStream
+    // first try using FileInputStream
     if (sockStream instanceof FileInputStream) {
       try {
         fd = ((FileInputStream)sockStream).getFD();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7b3c8cb4/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
old mode 100644
new mode 100755
index bf44cd3..6e949b2
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
@@ -74,7 +74,6 @@ import com.gemstone.gemfire.internal.ByteArrayDataInput;
 import com.gemstone.gemfire.internal.DSFIDFactory;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.SystemTimer.SystemTimerTask;
 import com.gemstone.gemfire.internal.Version;
@@ -1279,7 +1278,7 @@ public class Connection implements Runnable {
         int connectTime = getP2PConnectTimeout();; 
 
         try {
-          SocketUtils.connect(channel.socket(), addr, connectTime);
+          channel.socket().connect(addr, connectTime);
         } catch (NullPointerException e) {
           // bug #45044 - jdk 1.7 sometimes throws an NPE here
           ConnectException c = new ConnectException("Encountered bug #45044 - retrying");
@@ -1330,7 +1329,7 @@ public class Connection implements Runnable {
         s.setKeepAlive(SocketCreator.ENABLE_TCP_KEEP_ALIVE);
         setReceiveBufferSize(s, SMALL_BUFFER_SIZE);
         setSendBufferSize(s);
-        SocketUtils.connect(s, addr, 0);
+        s.connect(addr, 0);
       }
     }
     if (logger.isDebugEnabled()) {


[25/32] incubator-geode git commit: GEODE-1162: convert all CLI command DUnit tests to JUnit 4

Posted by ji...@apache.org.
GEODE-1162: convert all CLI command DUnit tests to JUnit 4

* GEODE-1161: cleanup VM and AsyncInvocation in DUnit
* define FlakyTest JUnit category
* define SecurityTest JUnit category and apply to all security tests (probably missed a few)
* create category test to discover JUnit behavior


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

Branch: refs/heads/feature/GEODE-17-2
Commit: f2d5969efb410011e760399765a02d07dbe17ed7
Parents: 7b3c8cb
Author: Kirk Lund <kl...@apache.org>
Authored: Thu Apr 14 13:45:26 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Thu Apr 14 13:45:26 2016 -0700

----------------------------------------------------------------------
 .../LauncherLifecycleCommandsDUnitTest.java     | 149 ++++--
 .../SharedConfigurationEndToEndDUnitTest.java   |  18 +-
 .../gemfire/internal/lang/ThrowableUtils.java   | 101 ++++
 .../query/dunit/SelectStarQueryDUnitTest.java   | 216 ++++----
 .../gemfire/cache30/MultiVMRegionTestCase.java  |  13 +-
 .../cache/partitioned/ShutdownAllDUnitTest.java |  10 +-
 .../PersistentRecoveryOrderDUnitTest.java       |   6 +-
 .../internal/lang/ThrowableUtilsTest.java       | 242 +++++++++
 .../AbstractCommandsSupportJUnitTest.java       |  16 +-
 .../cli/commands/CliCommandTestBase.java        |  57 +-
 .../cli/commands/ConfigCommandsDUnitTest.java   |  85 +--
 ...eateAlterDestroyRegionCommandsDUnitTest.java | 121 +++--
 .../cli/commands/DeployCommandsDUnitTest.java   |  33 +-
 .../commands/DiskStoreCommandsDUnitTest.java    |  83 +--
 .../commands/DiskStoreCommandsJUnitTest.java    |  19 +-
 .../cli/commands/FunctionCommandsDUnitTest.java |  97 ++--
 .../commands/GemfireDataCommandsDUnitTest.java  | 267 +++++-----
 ...WithCacheLoaderDuringCacheMissDUnitTest.java |  20 +-
 .../HTTPServiceSSLSupportJUnitTest.java         |  12 +-
 .../cli/commands/IndexCommandsDUnitTest.java    |  31 +-
 .../cli/commands/IndexCommandsJUnitTest.java    |  18 +-
 ...stAndDescribeDiskStoreCommandsDUnitTest.java |  23 +-
 .../ListAndDescribeRegionDUnitTest.java         |  50 +-
 .../cli/commands/ListIndexCommandDUnitTest.java |  29 +-
 .../cli/commands/MemberCommandsDUnitTest.java   |  60 ++-
 .../MiscellaneousCommandsDUnitTest.java         |  84 +--
 ...laneousCommandsExportLogsPart1DUnitTest.java |  37 +-
 ...laneousCommandsExportLogsPart2DUnitTest.java |  38 +-
 ...laneousCommandsExportLogsPart3DUnitTest.java |  39 +-
 ...laneousCommandsExportLogsPart4DUnitTest.java |  36 +-
 .../cli/commands/QueueCommandsDUnitTest.java    |  41 +-
 .../SharedConfigurationCommandsDUnitTest.java   |  55 +-
 .../cli/commands/ShellCommandsDUnitTest.java    |  47 +-
 .../cli/commands/ShowDeadlockDUnitTest.java     |  56 +-
 .../cli/commands/ShowMetricsDUnitTest.java      |  71 +--
 .../cli/commands/ShowStackTraceDUnitTest.java   |  43 +-
 .../cli/commands/UserCommandsDUnitTest.java     |  24 +-
 .../security/ClientAuthenticationDUnitTest.java |   6 +-
 .../ClientAuthenticationPart2DUnitTest.java     |   6 +-
 .../security/ClientAuthenticationTestCase.java  |   3 +
 .../security/ClientAuthorizationDUnitTest.java  |  10 +-
 .../security/ClientAuthorizationTestCase.java   |   2 +-
 .../security/ClientMultiUserAuthzDUnitTest.java |  10 +-
 .../DeltaClientAuthorizationDUnitTest.java      |  10 +-
 .../DeltaClientPostAuthorizationDUnitTest.java  |   8 +-
 .../security/GemFireSecurityExceptionTest.java  |   6 +-
 .../security/NotAuthorizedExceptionTest.java    |   6 +-
 .../security/P2PAuthenticationDUnitTest.java    |  10 +-
 .../gemfire/security/SecurityTestUtils.java     |  13 +-
 .../generator/AuthzCredentialGenerator.java     |  15 +-
 .../security/generator/CredentialGenerator.java |   3 +-
 .../generator/DummyCredentialGenerator.java     |   6 +-
 .../generator/LdapUserCredentialGenerator.java  |  15 +-
 .../generator/PKCSCredentialGenerator.java      |   8 +-
 .../generator/SSLCredentialGenerator.java       |   9 +-
 .../UserPasswordWithExtraPropsAuthInit.java     |   6 +-
 .../generator/XmlAuthzCredentialGenerator.java  |  10 +-
 .../security/templates/DummyAuthorization.java  |   8 +-
 .../templates/LdapUserAuthenticator.java        |   3 +-
 .../security/templates/PKCSAuthInit.java        |   3 +-
 .../security/templates/PKCSAuthenticator.java   |   3 +-
 .../security/templates/PKCSPrincipalTest.java   |   9 +-
 .../templates/UserPasswordAuthInit.java         |   4 +-
 .../templates/UsernamePrincipalTest.java        |   9 +-
 .../security/templates/XmlAuthorization.java    |  19 +-
 .../security/templates/XmlErrorHandler.java     |   5 +-
 .../gemfire/test/dunit/AsyncInvocation.java     | 511 +++++++++++++-----
 .../gemfire/test/dunit/DistributedTestCase.java |   8 +-
 .../test/dunit/DistributedTestUtils.java        |   2 +-
 .../gemfire/test/dunit/ThreadUtils.java         |  13 +-
 .../com/gemstone/gemfire/test/dunit/VM.java     | 521 +++++++++----------
 .../com/gemstone/gemfire/test/dunit/Wait.java   |   2 +-
 .../cache/internal/JUnit4CacheTestCase.java     |   4 +-
 .../internal/JUnit3DistributedTestCase.java     |   9 +-
 .../internal/JUnit4DistributedTestCase.java     |  17 +-
 .../internal/tests/JUnit3BasicDUnitTest.java    | 189 -------
 .../JUnit3GetDefaultDiskStoreNameDUnitTest.java |  66 ---
 .../tests/JUnit3GetTestMethodNameDUnitTest.java |  53 --
 .../dunit/internal/tests/JUnit3VMDUnitTest.java | 192 -------
 .../internal/tests/JUnit4BasicDUnitTest.java    | 202 -------
 .../JUnit4GetDefaultDiskStoreNameDUnitTest.java |  65 ---
 .../tests/JUnit4GetTestMethodNameDUnitTest.java |  52 --
 .../dunit/internal/tests/JUnit4VMDUnitTest.java | 200 -------
 .../DistributedRestoreSystemProperties.java     |   3 +-
 .../gemfire/test/dunit/rules/RemoteInvoker.java |   3 +-
 .../gemfire/test/dunit/standalone/ChildVM.java  |   5 +-
 .../test/dunit/standalone/DUnitLauncher.java    |  10 +-
 .../test/dunit/standalone/RemoteDUnitVM.java    |   5 +-
 .../test/dunit/standalone/RemoteDUnitVMIF.java  |   4 +-
 .../test/dunit/tests/BasicDUnitTest.java        | 218 ++++----
 .../test/dunit/tests/JUnit4BasicDUnitTest.java  | 234 +++++++++
 .../JUnit4GetDefaultDiskStoreNameDUnitTest.java |  66 +++
 .../tests/JUnit4GetTestMethodNameDUnitTest.java |  53 ++
 ...ingGetPropertiesDisconnectsAllDUnitTest.java |  57 ++
 .../test/dunit/tests/JUnit4VMDUnitTest.java     | 201 +++++++
 .../cli/commands/ClientCommandsDUnitTest.java   | 199 +++----
 .../DurableClientCommandsDUnitTest.java         |  53 +-
 .../ClientAuthorizationTwoDUnitTest.java        |   8 +-
 .../security/ClientAuthzObjectModDUnitTest.java |   8 +-
 .../ClientCQPostAuthorizationDUnitTest.java     |   8 +-
 .../ClientPostAuthorizationDUnitTest.java       |  14 +-
 .../gemfire/security/MultiUserAPIDUnitTest.java |   8 +-
 .../MultiUserDurableCQAuthzDUnitTest.java       |   8 +-
 .../test/junit/categories/FlakyTest.java        |  24 +
 .../test/junit/categories/SecurityTest.java     |  23 +
 .../gemfire/test/junit/rules/RetryRule.java     |   2 +-
 .../test/junit/categories/CategoryOne.java      |  20 +
 .../test/junit/categories/CategoryTest.java     | 169 ++++++
 .../test/junit/categories/CategoryTwo.java      |  20 +
 .../wan/wancommand/WANCommandTestBase.java      |  35 +-
 ...anCommandCreateGatewayReceiverDUnitTest.java |  45 +-
 .../WanCommandCreateGatewaySenderDUnitTest.java |  45 +-
 ...WanCommandGatewayReceiverStartDUnitTest.java |  39 +-
 .../WanCommandGatewayReceiverStopDUnitTest.java |  37 +-
 .../WanCommandGatewaySenderStartDUnitTest.java  |  44 +-
 .../WanCommandGatewaySenderStopDUnitTest.java   |  39 +-
 .../wan/wancommand/WanCommandListDUnitTest.java |  52 +-
 .../WanCommandPauseResumeDUnitTest.java         | 138 ++---
 .../wancommand/WanCommandStatusDUnitTest.java   |  79 +--
 .../ClusterConfigurationDUnitTest.java          |  64 ++-
 120 files changed, 3731 insertions(+), 2987 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
index e41e118..4553140 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
@@ -16,8 +16,43 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
 import static com.gemstone.gemfire.test.dunit.Wait.*;
 
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.lang.management.ManagementFactory;
+import java.net.InetAddress;
+import java.nio.charset.Charset;
+import java.text.DateFormat;
+import java.text.MessageFormat;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.TimeUnit;
+import javax.management.MBeanServerConnection;
+import javax.management.ObjectName;
+import javax.management.Query;
+import javax.management.QueryExp;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+
+import org.junit.FixMethodOrder;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runners.MethodSorters;
+
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
@@ -46,38 +81,7 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
-
-import org.junit.FixMethodOrder;
-import org.junit.Ignore;
-import org.junit.runners.MethodSorters;
-
-import javax.management.MBeanServerConnection;
-import javax.management.ObjectName;
-import javax.management.Query;
-import javax.management.QueryExp;
-import javax.management.remote.JMXConnector;
-import javax.management.remote.JMXConnectorFactory;
-import javax.management.remote.JMXServiceURL;
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
-import java.nio.charset.Charset;
-import java.text.DateFormat;
-import java.text.MessageFormat;
-import java.text.SimpleDateFormat;
-import java.util.Calendar;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.ConcurrentLinkedDeque;
-import java.util.concurrent.TimeUnit;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * The LauncherLifecycleCommandsDUnitTest class is a test suite of integration tests testing the contract and
@@ -95,6 +99,7 @@ import java.util.concurrent.TimeUnit;
  * @see com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder
  * @since 7.0
  */
+@Category(DistributedTest.class)
 @FixMethodOrder(MethodSorters.NAME_ASCENDING)
 public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
 
@@ -104,10 +109,6 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
 
   private final Queue<Integer> processIds = new ConcurrentLinkedDeque<>();
 
-  public LauncherLifecycleCommandsDUnitTest(final String testName) {
-    super(testName);
-  }
-
   protected static String getMemberId(final int jmxManagerPort, final String memberName) throws Exception {
     return getMemberId(InetAddress.getLocalHost().getHostName(), jmxManagerPort, memberName);
   }
@@ -296,6 +297,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void test000StartLocatorCapturesOutputOnError() throws IOException {
     final int locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
 
@@ -356,6 +358,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     return pid;
   }
 
+  @Test
   public void test001StartLocatorFailsFastOnMissingGemFirePropertiesFile() {
     String gemfirePropertiesPathname = "/path/to/missing/gemfire.properties";
 
@@ -381,6 +384,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
             gemfirePropertiesPathname)));
   }
 
+  @Test
   public void test002StartLocatorFailsFastOnMissingGemFireSecurityPropertiesFile() {
     String gemfireSecurityPropertiesPathname = "/path/to/missing/gemfire-security.properties";
 
@@ -406,6 +410,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
             gemfireSecurityPropertiesPathname)));
   }
 
+  @Test
   public void test003StartServerFailsFastOnMissingCacheXmlFile() {
     String cacheXmlPathname = "/path/to/missing/cache.xml";
 
@@ -425,6 +430,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
         resultString.contains(MessageFormat.format(CliStrings.CACHE_XML_NOT_FOUND_MESSAGE, cacheXmlPathname)));
   }
 
+  @Test
   public void test004StartServerFailsFastOnMissingGemFirePropertiesFile() {
     String gemfirePropertiesFile = "/path/to/missing/gemfire.properties";
 
@@ -445,6 +451,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
             gemfirePropertiesFile)));
   }
 
+  @Test
   public void test005StartServerFailsFastOnMissingGemFireSecurityPropertiesFile() {
     String gemfireSecuritiesPropertiesFile = "/path/to/missing/gemfire-securities.properties";
 
@@ -465,6 +472,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
             gemfireSecuritiesPropertiesFile)));
   }
 
+  @Test
   public void test006StartLocatorInRelativeDirectory() {
     final int locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
 
@@ -500,6 +508,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void test007StatusLocatorUsingMemberNameIDWhenGfshIsNotConnected() {
     CommandResult result = executeCommand(CliStrings.STATUS_LOCATOR + " --name=" + getTestMethodName());
 
@@ -509,6 +518,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
         StringUtils.trim(toString(result)));
   }
 
+  @Test
   public void test008StatusLocatorUsingMemberName() {
     final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
 
@@ -571,6 +581,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void test009StatusLocatorUsingMemberId() throws Exception {
     final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
 
@@ -627,6 +638,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void test010StopLocatorUsingMemberNameIDWhenGfshIsNotConnected() {
     CommandResult result = executeCommand(CliStrings.STOP_LOCATOR + " --name=" + getTestMethodName());
 
@@ -636,6 +648,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
         StringUtils.trim(toString(result)));
   }
 
+  @Test
   public void test011StopLocatorUsingMemberName() {
     final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
 
@@ -723,6 +736,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
   }
 
   // @see Trac Bug # 46760
+  @Test
   public void test012StopLocatorUsingMemberId() throws Exception {
     final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
 
@@ -799,6 +813,69 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     assertEquals(Status.NOT_RESPONDING, locatorState.getStatus());
   }
 
+  @Ignore("Disabled until GEODE-1025, SGF-476 are resolved")
+  @Test
+  public void test013StartServerWithSpring() {
+    String pathname = (getClass().getSimpleName() + "_" + getTestMethodName());
+    File workingDirectory = new File(pathname);
+
+    assertTrue(workingDirectory.isDirectory() || workingDirectory.mkdir());
+
+    CommandStringBuilder command = new CommandStringBuilder(CliStrings.START_SERVER);
+
+    command.addOption(CliStrings.START_SERVER__NAME, getClass().getSimpleName().concat("_").concat(getTestMethodName()));
+    command.addOption(CliStrings.START_SERVER__USE_CLUSTER_CONFIGURATION, Boolean.FALSE.toString());
+    command.addOption(CliStrings.START_SERVER__LOG_LEVEL, "config");
+    command.addOption(CliStrings.START_SERVER__INCLUDE_SYSTEM_CLASSPATH);
+    command.addOption(CliStrings.START_SERVER__DISABLE_DEFAULT_SERVER);
+    command.addOption(CliStrings.START_SERVER__DIR, pathname);
+    command.addOption(CliStrings.START_SERVER__SPRING_XML_LOCATION, "spring/spring-gemfire-context.xml");
+
+    CommandResult result = executeCommand(command.toString());
+
+    assertNotNull(result);
+    assertEquals(Result.Status.OK, result.getStatus());
+
+    final ServerLauncher springGemFireServer = new ServerLauncher.Builder().setCommand(
+        ServerLauncher.Command.STATUS).setWorkingDirectory(
+        IOUtils.tryGetCanonicalPathElseGetAbsolutePath(workingDirectory)).build();
+
+    assertNotNull(springGemFireServer);
+
+    ServerState serverState = springGemFireServer.status();
+    
+    assertNotNull(serverState);
+    assertEquals(Status.ONLINE, serverState.getStatus());
+    
+    //Ensure the member name is what is set through spring
+    String logFile = serverState.getLogFile();
+    assertTrue("Log file name was not configured from spring context: " + logFile, logFile.contains("spring_server.log"));
+
+    // Now that the GemFire Server bootstrapped with Spring started up OK, stop it!
+    stopServer(springGemFireServer.getWorkingDirectory());
+
+    WaitCriterion waitCriteria = new WaitCriterion() {
+      @Override
+      public boolean done() {
+        ServerState serverState = springGemFireServer.status();
+        return (serverState != null && Status.NOT_RESPONDING.equals(serverState.getStatus()));
+      }
+
+      @Override
+      public String description() {
+        return "wait for the Locator to stop; the Locator will no longer respond after it stops";
+      }
+    };
+
+    waitForCriterion(waitCriteria, TimeUnit.SECONDS.toMillis(15), TimeUnit.SECONDS.toMillis(5), true);
+
+    serverState = springGemFireServer.status();
+
+    assertNotNull(serverState);
+    assertEquals(Status.NOT_RESPONDING, serverState.getStatus());
+  }
+
+  @Test
   public void test014GemFireServerJvmProcessTerminatesOnOutOfMemoryError() throws Exception {
     int ports[] = AvailablePortHelper.getRandomAvailableTCPPorts(2);
     final int serverPort = ports[0];

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
index edd056b..e1093a2 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.management.internal.configuration;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
 import static com.gemstone.gemfire.test.dunit.Wait.*;
 
 import com.gemstone.gemfire.cache.Cache;
@@ -40,15 +42,16 @@ import com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 import org.apache.commons.io.FileUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import java.io.File;
 import java.io.IOException;
@@ -59,6 +62,7 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 
+@Category(DistributedTest.class)
 public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
   private static final int TIMEOUT = 10000;
   private static final int INTERVAL = 500;
@@ -69,11 +73,6 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
   public static Set<String> serverNames = new HashSet<String>();
   public static Set<String> jarFileNames = new HashSet<String>();
 
-  public SharedConfigurationEndToEndDUnitTest(String name) {
-    super(name);
-    // TODO Auto-generated constructor stub
-  }
-
   private static final long serialVersionUID = -2276690105585944041L;
 
   public Set<String> startServers(HeadlessGfsh gfsh, String locatorString, int numServers, String serverNamePrefix, int startNum) throws ClassNotFoundException, IOException {
@@ -93,6 +92,7 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
     return serverNames;
   }
 
+  @Test
   public void testStartServerAndExecuteCommands() throws InterruptedException, ClassNotFoundException, IOException, ExecutionException {
     IgnoredException.addIgnoredException("EntryDestroyedException");
     Object[] result = setup();
@@ -117,7 +117,7 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
 
 
     //shutdown everything
-    LogWriterUtils.getLogWriter().info("Shutting down all the members");
+    getLogWriter().info("Shutting down all the members");
     shutdownAll();
     deleteSavedJarFiles();
   }
@@ -147,7 +147,7 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
 
   protected void executeAndVerifyCommand(String commandString) {
     CommandResult cmdResult = executeCommand(commandString);
-    LogWriterUtils.getLogWriter().info("Command Result : \n" + commandResultToString(cmdResult));
+    getLogWriter().info("Command Result : \n" + commandResultToString(cmdResult));
     assertEquals(Status.OK, cmdResult.getStatus());
     assertFalse(cmdResult.failedToPersist());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/main/java/com/gemstone/gemfire/internal/lang/ThrowableUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/lang/ThrowableUtils.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/lang/ThrowableUtils.java
new file mode 100644
index 0000000..eb3188b
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/lang/ThrowableUtils.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.lang;
+
+/**
+ * The ThrowableUtils class is an abstract utility class for working with {@code Throwable}s.
+ * <p/>
+ * @see java.lang.Throwable
+ */
+public abstract class ThrowableUtils {
+
+  /**
+   * Get the root cause of a {@link Throwable}. Returns the specified
+   * {@code throwable} if its {@code getCause} returns null.
+   *
+   * @param  throwable the {@code Throwable} to get the root cause of.
+   * @return the root cause of the specified {@code throwable}.
+   *
+   * @throws NullPointerException if {@code throwable} is null
+   */
+  public static Throwable getRootCause(Throwable throwable) {
+    if (throwable.getCause() == null) {
+      return throwable;
+    }
+
+    Throwable cause;
+    while ((cause = throwable.getCause()) != null) {
+      throwable = cause;
+    }
+    return throwable;
+  }
+
+  /**
+   * Returns true if the {@link Throwable} or any of its causes as returned
+   * by {@code getCause()} are an instance of the specified subclass of
+   * {@code Throwable}.
+   *
+   * @param  throwable the {@code Throwable} to check the causes of.
+   * @param  causeClass the subclass of {@code Throwable} to check for.
+   * @return true if any cause of {@code throwable} is an instance of
+   *         {@code causeClass}.
+   *
+   * @throws NullPointerException if {@code throwable} is null
+   */
+  public static boolean hasCauseType(Throwable throwable, Class<? extends Throwable> causeClass) {
+    if (causeClass.isInstance(throwable)) {
+      return true;
+    }
+
+    Throwable cause;
+    while ((cause = throwable.getCause()) != null) {
+      throwable = cause;
+      if (causeClass.isInstance(throwable)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Returns true if the {@link Throwable} or any of its causes contain the
+   * specified {@code message}.
+   *
+   * @param  throwable the {@code Throwable} to check the causes of.
+   * @param  message the {@code Throwable} message to check for.
+   * @return true if any cause of {@code throwable} contains the specified
+   *         {@code message}.
+   *
+   * @throws NullPointerException if {@code throwable} is null
+   */
+  public static boolean hasCauseMessage(Throwable throwable, String message) {
+    if (throwable.getMessage().contains(message)) {
+      return true;
+    }
+
+    Throwable cause;
+    while ((cause = throwable.getCause()) != null) {
+      throwable = cause;
+      if (throwable.getMessage().contains(message)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/SelectStarQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/SelectStarQueryDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/SelectStarQueryDUnitTest.java
index b92426a..11b76de 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/SelectStarQueryDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/SelectStarQueryDUnitTest.java
@@ -14,11 +14,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.cache.query.dunit;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Invoke.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.NetworkUtils.*;
+
 import java.io.Serializable;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.client.ClientCache;
@@ -35,28 +42,25 @@ import com.gemstone.gemfire.cache.query.internal.QueryObserverAdapter;
 import com.gemstone.gemfire.cache.query.internal.QueryObserverHolder;
 import com.gemstone.gemfire.cache.query.internal.StructImpl;
 import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.VMCachedDeserializable;
 import com.gemstone.gemfire.pdx.PdxInstance;
-import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
-import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Test for #44807 to eliminate unnecessary serialization/deserialization in
  * select * queries
- * 
- * 
  */
-public class SelectStarQueryDUnitTest extends CacheTestCase {
-  public SelectStarQueryDUnitTest(String name) {
-    super(name);
-  }
+@Category(DistributedTest.class)
+public class SelectStarQueryDUnitTest extends JUnit4CacheTestCase {
+
+  /** Used for saving & restoring oldObserver without serialization */
+  private static volatile QueryObserver oldObserver;
 
   private final String regName = "exampleRegion";
   private final String regName2 = "exampleRegion2";
@@ -87,6 +91,12 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
           + regName2 + " q, q.positions.values pos", };
   private final int[] resultSize2 = { 400, 5, 400, 400, 1, 400, 400, 400, 1600 };
 
+  @Override
+  public final void preTearDownCacheTestCase() throws Exception {
+    invokeInEveryVM(() -> oldObserver = null);
+  }
+
+  @Test
   public void testSelectStarQueryForPartitionedRegion() throws Exception {
     final Host host = Host.getHost(0);
     final VM server1 = host.getVM(0);
@@ -103,13 +113,11 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     final int port2 = startPartitionedCacheServer(server2, portfolios);
     final int port3 = startPartitionedCacheServer(server3, portfolios);
 
-    final QueryObserver oldObserver = (QueryObserver) server1
-        .invoke(new SerializableCallable("Set observer") {
+    server1.invoke(new SerializableCallable("Set observer") {
           @Override
           public Object call() throws Exception {
-            QueryObserver observer = QueryObserverHolder
-                .setInstance(new QueryResultTrackingObserver());
-            return observer;
+            oldObserver = QueryObserverHolder.setInstance(new QueryResultTrackingObserver());
+            return null;
           }
         });
 
@@ -118,9 +126,9 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
-        cf.addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port1);
-        cf.addPoolServer(NetworkUtils.getServerHostName(server2.getHost()), port2);
-        cf.addPoolServer(NetworkUtils.getServerHostName(server3.getHost()), port3);
+        cf.addPoolServer(getServerHostName(server1.getHost()), port1);
+        cf.addPoolServer(getServerHostName(server2.getHost()), port2);
+        cf.addPoolServer(getServerHostName(server3.getHost()), port3);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
             .create(regName);
@@ -144,14 +152,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     client.invoke(new SerializableCallable("Query") {
       @Override
       public Object call() throws Exception {
-        LogWriterUtils.getLogWriter().info("Querying remotely from client");
+        getLogWriter().info("Querying remotely from client");
         QueryService localQS = null;
         QueryService remoteQS = null;
         try {
           localQS = ((ClientCache) getCache()).getLocalQueryService();
           remoteQS = ((ClientCache) getCache()).getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults[][] sr = new SelectResults[1][2];
         SelectResults res = null;
@@ -163,7 +171,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             sr[0][1] = res;
             CacheUtils.compareResultsOfWithAndWithoutIndex(sr);
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -217,14 +225,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
         try {
           qs = getCache().getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         for (int i = 0; i < queries.length; i++) {
           try {
             res = (SelectResults) qs.newQuery(queries[i]).execute();
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -268,6 +276,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     closeCache(server3);
   }
 
+  @Test
   public void testSelectStarQueryForReplicatedRegion() throws Exception {
     final Host host = Host.getHost(0);
     final VM server1 = host.getVM(1);
@@ -275,13 +284,11 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     // create servers and regions
     final int port1 = startReplicatedCacheServer(server1);
 
-    final QueryObserver oldObserver = (QueryObserver) server1
-        .invoke(new SerializableCallable("Set observer") {
+    server1.invoke(new SerializableCallable("Set observer") {
           @Override
           public Object call() throws Exception {
-            QueryObserver observer = QueryObserverHolder
-                .setInstance(new QueryResultTrackingObserver());
-            return observer;
+            oldObserver = QueryObserverHolder.setInstance(new QueryResultTrackingObserver());
+            return null;
           }
         });
 
@@ -290,7 +297,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
-        cf.addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port1);
+        cf.addPoolServer(getServerHostName(server1.getHost()), port1);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
             .create(regName);
@@ -318,14 +325,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     client.invoke(new SerializableCallable("Query") {
       @Override
       public Object call() throws Exception {
-        LogWriterUtils.getLogWriter().info("Querying remotely from client");
+        getLogWriter().info("Querying remotely from client");
         QueryService localQS = null;
         QueryService remoteQS = null;
         try {
           localQS = ((ClientCache) getCache()).getLocalQueryService();
           remoteQS = ((ClientCache) getCache()).getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         SelectResults[][] sr = new SelectResults[1][2];
@@ -340,7 +347,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             sr[0][1] = res;
             CacheUtils.compareResultsOfWithAndWithoutIndex(sr);
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + multipleRegionQueries[i], e);
+            fail("Error executing query: " + multipleRegionQueries[i], e);
           }
           assertEquals(resultSize2[i], res.size());
           if (i == 4) {
@@ -397,7 +404,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
         try {
           qs = getCache().getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         for (int i = 0; i < multipleRegionQueries.length; i++) {
@@ -405,7 +412,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             res = (SelectResults) qs.newQuery(multipleRegionQueries[i])
                 .execute();
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + multipleRegionQueries[i], e);
+            fail("Error executing query: " + multipleRegionQueries[i], e);
           }
           assertEquals(resultSize2[i], res.size());
           if (i == 4) {
@@ -450,6 +457,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     closeCache(server1);
   }
 
+  @Test
   public void testByteArrayReplicatedRegion() throws Exception {
     final Host host = Host.getHost(0);
     final VM server1 = host.getVM(0);
@@ -475,13 +483,11 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
       }
     });
 
-    final QueryObserver oldObserver = (QueryObserver) server1
-        .invoke(new SerializableCallable("Set observer") {
+    server1.invoke(new SerializableCallable("Set observer") {
           @Override
           public Object call() throws Exception {
-            QueryObserver observer = QueryObserverHolder
-                .setInstance(new QueryResultTrackingObserver());
-            return observer;
+            oldObserver = QueryObserverHolder.setInstance(new QueryResultTrackingObserver());
+            return null;
           }
         });
 
@@ -490,7 +496,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
-        cf.addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port);
+        cf.addPoolServer(getServerHostName(server1.getHost()), port);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
             .create(regName);
@@ -514,14 +520,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     client.invoke(new SerializableCallable("Query") {
       @Override
       public Object call() throws Exception {
-        LogWriterUtils.getLogWriter().info("Querying remotely from client");
+        getLogWriter().info("Querying remotely from client");
         QueryService localQS = null;
         QueryService remoteQS = null;
         try {
           localQS = ((ClientCache) getCache()).getLocalQueryService();
           remoteQS = ((ClientCache) getCache()).getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         SelectResults[][] sr = new SelectResults[1][2];
@@ -534,7 +540,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             sr[0][1] = res;
             CacheUtils.compareResultsOfWithAndWithoutIndex(sr);
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -585,14 +591,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
         try {
           qs = getCache().getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         for (int i = 0; i < 6; i++) {
           try {
             res = (SelectResults) qs.newQuery(queries[i]).execute();
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -631,6 +637,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     closeCache(server1);
   }
 
+  @Test
   public void testByteArrayPartitionedRegion() throws Exception {
     final Host host = Host.getHost(0);
     final VM server1 = host.getVM(0);
@@ -648,13 +655,11 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     final int port2 = startPartitionedCacheServer(server2, objs);
     final int port3 = startPartitionedCacheServer(server3, objs);
 
-    final QueryObserver oldObserver = (QueryObserver) server1
-        .invoke(new SerializableCallable("Set observer") {
+    server1.invoke(new SerializableCallable("Set observer") {
           @Override
           public Object call() throws Exception {
-            QueryObserver observer = QueryObserverHolder
-                .setInstance(new QueryResultTrackingObserver());
-            return observer;
+            oldObserver = QueryObserverHolder.setInstance(new QueryResultTrackingObserver());
+            return null;
           }
         });
 
@@ -663,9 +668,9 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
-        cf.addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port1);
-        cf.addPoolServer(NetworkUtils.getServerHostName(server2.getHost()), port2);
-        cf.addPoolServer(NetworkUtils.getServerHostName(server3.getHost()), port3);
+        cf.addPoolServer(getServerHostName(server1.getHost()), port1);
+        cf.addPoolServer(getServerHostName(server2.getHost()), port2);
+        cf.addPoolServer(getServerHostName(server3.getHost()), port3);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
             .create(regName);
@@ -689,14 +694,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     client.invoke(new SerializableCallable("Query") {
       @Override
       public Object call() throws Exception {
-        LogWriterUtils.getLogWriter().info("Querying remotely from client");
+        getLogWriter().info("Querying remotely from client");
         QueryService localQS = null;
         QueryService remoteQS = null;
         try {
           localQS = ((ClientCache) getCache()).getLocalQueryService();
           remoteQS = ((ClientCache) getCache()).getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         SelectResults[][] sr = new SelectResults[1][2];
@@ -709,7 +714,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             sr[0][1] = res;
             CacheUtils.compareResultsOfWithAndWithoutIndex(sr);
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -760,14 +765,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
         try {
           qs = getCache().getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         for (int i = 0; i < 6; i++) {
           try {
             res = (SelectResults) qs.newQuery(queries[i]).execute();
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -808,6 +813,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
 
   }
 
+  @Test
   public void testSelectStarQueryForIndexes() throws Exception {
     final Host host = Host.getHost(0);
     final VM server1 = host.getVM(0);
@@ -819,7 +825,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
-        cf.addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port1);
+        cf.addPoolServer(getServerHostName(server1.getHost()), port1);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
             .create(regName);
@@ -852,7 +858,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
           qs = getCache().getQueryService();
           qs.createIndex("status", "status", "/" + regName);
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
 
         return null;
@@ -863,14 +869,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     client.invoke(new SerializableCallable("Query") {
       @Override
       public Object call() throws Exception {
-        LogWriterUtils.getLogWriter().info("Querying remotely from client");
+        getLogWriter().info("Querying remotely from client");
         QueryService localQS = null;
         QueryService remoteQS = null;
         try {
           localQS = ((ClientCache) getCache()).getLocalQueryService();
           remoteQS = ((ClientCache) getCache()).getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         SelectResults[][] sr = new SelectResults[1][2];
@@ -885,7 +891,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             sr[0][1] = res;
             CacheUtils.compareResultsOfWithAndWithoutIndex(sr);
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + multipleRegionQueries[i], e);
+            fail("Error executing query: " + multipleRegionQueries[i], e);
           }
           assertEquals(resultSize2[i], res.size());
           if (i == 4) {
@@ -928,7 +934,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
           qs = getCache().getQueryService();
           qs.createIndex("status", "status", "/" + regName2);
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
 
         return null;
@@ -939,14 +945,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     client.invoke(new SerializableCallable("Query") {
       @Override
       public Object call() throws Exception {
-        LogWriterUtils.getLogWriter().info("Querying remotely from client");
+        getLogWriter().info("Querying remotely from client");
         QueryService localQS = null;
         QueryService remoteQS = null;
         try {
           localQS = ((ClientCache) getCache()).getLocalQueryService();
           remoteQS = ((ClientCache) getCache()).getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         SelectResults[][] sr = new SelectResults[1][2];
@@ -961,7 +967,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             sr[0][1] = res;
             CacheUtils.compareResultsOfWithAndWithoutIndex(sr);
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + multipleRegionQueries[i], e);
+            fail("Error executing query: " + multipleRegionQueries[i], e);
           }
           assertEquals(resultSize2[i], res.size());
           if (i == 4) {
@@ -999,6 +1005,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     closeCache(server1);
   }
 
+  @Test
   public void testSelectStarQueryForPdxObjects() throws Exception {
     final Host host = Host.getHost(0);
     final VM server1 = host.getVM(0);
@@ -1006,13 +1013,11 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     // create servers and regions
     final int port1 = startReplicatedCacheServer(server1);
 
-    final QueryObserver oldObserver = (QueryObserver) server1
-        .invoke(new SerializableCallable("Set observer") {
+    server1.invoke(new SerializableCallable("Set observer") {
           @Override
           public Object call() throws Exception {
-            QueryObserver observer = QueryObserverHolder
-                .setInstance(new QueryResultTrackingObserver());
-            return observer;
+            oldObserver = QueryObserverHolder.setInstance(new QueryResultTrackingObserver());
+            return null;
           }
         });
 
@@ -1021,7 +1026,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
-        cf.addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port1);
+        cf.addPoolServer(getServerHostName(server1.getHost()), port1);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
             .create(regName);
@@ -1045,14 +1050,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     client.invoke(new SerializableCallable("Query") {
       @Override
       public Object call() throws Exception {
-        LogWriterUtils.getLogWriter().info("Querying remotely from client");
+        getLogWriter().info("Querying remotely from client");
         QueryService localQS = null;
         QueryService remoteQS = null;
         try {
           localQS = ((ClientCache) getCache()).getLocalQueryService();
           remoteQS = ((ClientCache) getCache()).getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         SelectResults[][] sr = new SelectResults[1][2];
@@ -1065,7 +1070,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             sr[0][1] = res;
             CacheUtils.compareResultsOfWithAndWithoutIndex(sr);
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -1120,14 +1125,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
         try {
           qs = getCache().getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         for (int i = 0; i < queries.length; i++) {
           try {
             res = (SelectResults) qs.newQuery(queries[i]).execute();
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -1176,14 +1181,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
         try {
           qs = getCache().getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         for (int i = 0; i < queries.length; i++) {
           try {
             res = (SelectResults) qs.newQuery(queries[i]).execute();
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -1219,6 +1224,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     closeCache(server1);
   }
 
+  @Test
   public void testSelectStarQueryForPdxAndNonPdxObjects() throws Exception {
     final Host host = Host.getHost(0);
     final VM server1 = host.getVM(0);
@@ -1227,13 +1233,11 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     // put domain objects
     final int port1 = startReplicatedCacheServer(server1);
 
-    final QueryObserver oldObserver = (QueryObserver) server1
-        .invoke(new SerializableCallable("Set observer") {
+    server1.invoke(new SerializableCallable("Set observer") {
           @Override
           public Object call() throws Exception {
-            QueryObserver observer = QueryObserverHolder
-                .setInstance(new QueryResultTrackingObserver());
-            return observer;
+            oldObserver = QueryObserverHolder.setInstance(new QueryResultTrackingObserver());
+            return null;
           }
         });
 
@@ -1242,7 +1246,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
-        cf.addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port1);
+        cf.addPoolServer(getServerHostName(server1.getHost()), port1);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
             .create(regName);
@@ -1266,14 +1270,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     client.invoke(new SerializableCallable("Query") {
       @Override
       public Object call() throws Exception {
-        LogWriterUtils.getLogWriter().info("Querying remotely from client");
+        getLogWriter().info("Querying remotely from client");
         QueryService localQS = null;
         QueryService remoteQS = null;
         try {
           localQS = ((ClientCache) getCache()).getLocalQueryService();
           remoteQS = ((ClientCache) getCache()).getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         SelectResults[][] sr = new SelectResults[1][2];
@@ -1286,7 +1290,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             sr[0][1] = res;
             CacheUtils.compareResultsOfWithAndWithoutIndex(sr);
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -1343,14 +1347,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
         try {
           qs = getCache().getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         for (int i = 0; i < queries.length; i++) {
           try {
             res = (SelectResults) qs.newQuery(queries[i]).execute();
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -1399,14 +1403,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
         try {
           qs = getCache().getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         for (int i = 0; i < queries.length; i++) {
           try {
             res = (SelectResults) qs.newQuery(queries[i]).execute();
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -1443,8 +1447,8 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     closeCache(server1);
   }
 
-  public void testSelectStarQueryForPdxObjectsReadSerializedTrue()
-      throws Exception {
+  @Test
+  public void testSelectStarQueryForPdxObjectsReadSerializedTrue() throws Exception {
     final Host host = Host.getHost(0);
     final VM server1 = host.getVM(0);
     final VM client = host.getVM(3);
@@ -1469,7 +1473,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
       @Override
       public Object call() throws Exception {
         ClientCacheFactory cf = new ClientCacheFactory();
-        cf.addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port);
+        cf.addPoolServer(getServerHostName(server1.getHost()), port);
         ClientCache cache = getClientCache(cf);
         cache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY)
             .create(regName);
@@ -1485,14 +1489,14 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     client.invoke(new SerializableCallable("Query") {
       @Override
       public Object call() throws Exception {
-        LogWriterUtils.getLogWriter().info("Querying remotely from client");
+        getLogWriter().info("Querying remotely from client");
         QueryService localQS = null;
         QueryService remoteQS = null;
         try {
           localQS = ((ClientCache) getCache()).getLocalQueryService();
           remoteQS = ((ClientCache) getCache()).getQueryService();
         } catch (Exception e) {
-          Assert.fail("Exception getting query service ", e);
+          fail("Exception getting query service ", e);
         }
         SelectResults res = null;
         SelectResults[][] sr = new SelectResults[1][2];
@@ -1505,7 +1509,7 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
             sr[0][1] = res;
             CacheUtils.compareResultsOfWithAndWithoutIndex(sr);
           } catch (Exception e) {
-            Assert.fail("Error executing query: " + queries[i], e);
+            fail("Error executing query: " + queries[i], e);
           }
           assertEquals(resultSize[i], res.size());
           if (i == 3) {
@@ -1597,17 +1601,17 @@ public class SelectStarQueryDUnitTest extends CacheTestCase {
     });
   }
 
-  public class QueryResultTrackingObserver extends QueryObserverAdapter 
-      implements Serializable{
+  public class QueryResultTrackingObserver extends QueryObserverAdapter implements Serializable {
+
     private boolean isObjectSerialized = false;
+
     @Override
-    public void beforeIterationEvaluation(CompiledValue executer,
-        Object currentObject) {
+    public void beforeIterationEvaluation(CompiledValue executer, Object currentObject) {
       if (currentObject instanceof VMCachedDeserializable) {
-        LogWriterUtils.getLogWriter().fine("currentObject is serialized object");
+        getLogWriter().fine("currentObject is serialized object");
         isObjectSerialized = true;
       } else {
-        LogWriterUtils.getLogWriter().fine("currentObject is deserialized object");
+        getLogWriter().fine("currentObject is deserialized object");
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/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 8f4740c..68e50bd 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
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache30;
 
+import static com.gemstone.gemfire.internal.lang.ThrowableUtils.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
@@ -9047,16 +9049,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
     try {
       async.getResult();
     } catch (Throwable e) {
-      if (e.getCause() instanceof RMIException) {
-        Throwable e2 = e.getCause();
-        if (e2.getCause() instanceof AssertionFailedError &&
-            e2.getCause().getMessage().equals(expectedError)) {
-          failed=true;
-        }
-      }
-      if (!failed) {
-        com.gemstone.gemfire.test.dunit.Assert.fail("asyncInvocation 0 returned exception", e);
-      }
+      assertTrue(hasCauseMessage(e, expectedError));
     }
     return failed;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java
index a1ba6d2..426d0ca 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/ShutdownAllDUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.internal.cache.partitioned;
 
+import static com.gemstone.gemfire.internal.lang.ThrowableUtils.getRootCause;
+
 import java.io.IOException;
 import java.util.Set;
 import java.util.TreeSet;
@@ -556,12 +558,8 @@ public class ShutdownAllDUnitTest extends CacheTestCase {
     try {
       a0.getResult(MAX_WAIT);
       fail("should have received a cache closed exception");
-    } catch(Exception e) {
-      if(!(e.getCause() instanceof RMIException)) {
-        throw e;
-      }
-      RMIException cause = (RMIException) e.getCause();
-      if(!(cause.getCause() instanceof CacheClosedException)) {
+    } catch(AssertionError e) {
+      if(!CacheClosedException.class.isInstance(getRootCause(e))) {
         throw e;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
index 321fffe..4f0fcca 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.internal.cache.persistence;
 
+import static com.gemstone.gemfire.internal.lang.ThrowableUtils.*;
+
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.File;
@@ -1317,8 +1319,8 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
     try {
       async1.getResult();
       fail("Should have seen a CacheClosedException");
-    } catch (Exception e) {
-      if (! (e.getCause().getCause() instanceof CacheClosedException)) {
+    } catch (AssertionError e) {
+      if (!CacheClosedException.class.isInstance(getRootCause(e))) {
         throw e;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/internal/lang/ThrowableUtilsTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/lang/ThrowableUtilsTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/lang/ThrowableUtilsTest.java
new file mode 100644
index 0000000..08feb63
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/lang/ThrowableUtilsTest.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.lang;
+
+import static org.assertj.core.api.Assertions.*;
+import static com.googlecode.catchexception.CatchException.*;
+import static com.googlecode.catchexception.CatchException.caughtException;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for {@link ThrowableUtils}
+ */
+@Category(UnitTest.class)
+public class ThrowableUtilsTest {
+
+  @Test
+  public void getRootCauseOfNullShouldThrowNullPointerException() {
+    catchException(this).getRootCause(null);
+
+    assertThat((Exception)caughtException()).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void getRootCauseOfLeafExceptionShouldReturnSameInstance() {
+    Throwable exception = new Exception();
+    Throwable rootCause = getRootCause(exception);
+
+    assertThat(rootCause).isSameAs(exception);
+  }
+
+  @Test
+  public void getRootCauseOfExceptionShouldReturnCause() {
+    Throwable cause = new Exception();
+    Throwable rootCause = getRootCause(new Exception(cause));
+
+    assertThat(rootCause).isSameAs(cause);
+  }
+
+  @Test
+  public void getRootCauseOfExceptionTreeShouldReturnCause() {
+    Throwable cause = new Exception();
+    Throwable rootCause = getRootCause(new Exception(new Exception(cause)));
+
+    assertThat(rootCause).isSameAs(cause);
+  }
+
+  @Test
+  public void getRootCauseOfErrorTreeShouldReturnCause() {
+    Throwable cause = new Error();
+    Throwable rootCause = getRootCause(new Error(new Error(cause)));
+
+    assertThat(rootCause).isSameAs(cause);
+  }
+
+  @Test
+  public void hasCauseTypeOfNullClassShouldThrowNullPointerException() {
+    catchException(this).hasCauseType(new Exception(), null);
+
+    assertThat((Exception)caughtException()).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void hasCauseTypeOfNullThrowableShouldThrowNullPointerException() {
+    catchException(this).hasCauseType(null, Exception.class);
+
+    assertThat((Exception)caughtException()).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void hasCauseTypeOfNonMatchingShouldReturnFalse() {
+    assertThat(hasCauseType(new OneException(), OtherException.class)).isFalse();
+  }
+
+  @Test
+  public void hasCauseTypeOfSameClassShouldReturnTrue() {
+    assertThat(hasCauseType(new OneException(), OneException.class)).isTrue();
+  }
+
+  @Test
+  public void hasCauseTypeOfSuperClassShouldReturnFalse() {
+    assertThat(hasCauseType(new OneException(), SubException.class)).isFalse();
+  }
+
+  @Test
+  public void hasCauseTypeOfSubClassShouldReturnTrue() {
+    assertThat(hasCauseType(new SubException(), OneException.class)).isTrue();
+  }
+
+  @Test
+  public void hasCauseTypeOfWrappedClassShouldReturnTrue() {
+    assertThat(hasCauseType(new OneException(new TwoException()), TwoException.class)).isTrue();
+  }
+
+  @Test
+  public void hasCauseTypeOfWrappingClassShouldReturnTrue() {
+    assertThat(hasCauseType(new OneException(new TwoException()), OneException.class)).isTrue();
+  }
+
+  @Test
+  public void hasCauseTypeOfNestedClassShouldReturnTrue() {
+    assertThat(hasCauseType(new OneException(new TwoException(new OtherException())), OtherException.class)).isTrue();
+  }
+
+  @Test
+  public void hasCauseMessageForNullShouldThrowNullPointerException() {
+    catchException(this).hasCauseMessage(null, "message");
+
+    assertThat((Exception)caughtException()).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void hasCauseMessageOfNullShouldThrowNullPointerException() {
+    catchException(this).hasCauseMessage(new OneException(), null);
+
+    assertThat((Exception)caughtException()).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void hasCauseMessageForNullMessageShouldThrowNullPointerException() {
+    catchException(this).hasCauseMessage(new OneException((String)null), null);
+
+    assertThat((Exception)caughtException()).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void hasCauseMessageOfNonMatchingNullMessageShouldThrowNullPointerException() {
+    catchException(this).hasCauseMessage(new OneException("message"), null);
+
+    assertThat((Exception)caughtException()).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void hasCauseMessageOfEmptyMessageShouldReturnTrue() {
+    assertThat(hasCauseMessage(new OneException(""), "")).isTrue();
+  }
+
+  @Test
+  public void hasCauseMessageOfMatchingMessageShouldReturnTrue() {
+    assertThat(hasCauseMessage(new OneException("message"), "message")).isTrue();
+  }
+
+  @Test
+  public void hasCauseMessageOfNonMatchingMessageShouldReturnFalse() {
+    assertThat(hasCauseMessage(new OneException("non-matching"), "message")).isFalse();
+  }
+
+  @Test
+  public void hasCauseMessageOfContainedMessageShouldReturnTrue() {
+    assertThat(hasCauseMessage(new OneException("this is the message"), "message")).isTrue();
+  }
+
+  @Test
+  public void hasCauseMessageOfPartialMatchingMessageShouldReturnFalse() {
+    assertThat(hasCauseMessage(new OneException("message"), "this is the message")).isFalse();
+  }
+
+  public Throwable getRootCause(final Throwable throwable) {
+    return ThrowableUtils.getRootCause(throwable);
+  }
+
+  public boolean hasCauseType(final Throwable throwable, final Class<? extends Throwable> causeClass) {
+    return ThrowableUtils.hasCauseType(throwable, causeClass);
+  }
+
+  public boolean hasCauseMessage(final Throwable throwable, final String message) {
+    return ThrowableUtils.hasCauseMessage(throwable, message);
+  }
+
+  private static class OneException extends Exception {
+    public OneException() {
+    }
+    public OneException(String message) {
+      super(message);
+    }
+    public OneException(Throwable cause) {
+      super(cause);
+    }
+    public OneException(String message, Throwable cause) {
+      super(message, cause);
+    }
+  }
+
+  private static class SubException extends OneException {
+    public SubException() {
+    }
+    public SubException(String message) {
+      super(message);
+    }
+    public SubException(Throwable cause) {
+      super(cause);
+    }
+    public SubException(String message, Throwable cause) {
+      super(message, cause);
+    }
+  }
+
+  private static class TwoException extends Exception {
+    public TwoException() {
+    }
+    public TwoException(String message) {
+      super(message);
+    }
+    public TwoException(Throwable cause) {
+      super(cause);
+    }
+    public TwoException(String message, Throwable cause) {
+      super(message, cause);
+    }
+  }
+
+  private static class OtherException extends Exception {
+    public OtherException() {
+    }
+    public OtherException(String message) {
+      super(message);
+    }
+    public OtherException(Throwable cause) {
+      super(cause);
+    }
+    public OtherException(String message, Throwable cause) {
+      super(message, cause);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/AbstractCommandsSupportJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/AbstractCommandsSupportJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/AbstractCommandsSupportJUnitTest.java
index 8d9971e..2b2377b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/AbstractCommandsSupportJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/AbstractCommandsSupportJUnitTest.java
@@ -23,6 +23,14 @@ import java.io.StringWriter;
 import java.util.Collections;
 import java.util.Set;
 
+import org.jmock.Expectations;
+import org.jmock.Mockery;
+import org.jmock.lib.legacy.ClassImposteriser;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionService;
@@ -35,14 +43,6 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.util.MemberNotFoundException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-import org.jmock.Expectations;
-import org.jmock.Mockery;
-import org.jmock.lib.legacy.ClassImposteriser;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 /**
  * The AbstractCommandsSupportJUnitTest class is a test suite of test cases testing the contract and functionality
  * of the AbstractCommandsSupport class for implementing GemFire shell (Gfsh) commands.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
index 7d8dc55..55f5556 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
@@ -16,8 +16,22 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.management.ManagementService;
@@ -29,29 +43,14 @@ import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
-
-import util.TestException;
-
-import java.io.IOException;
-import java.io.PrintStream;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
+import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
 
 /**
  * Base class for all the CLI/gfsh command dunit tests.
- *
  */
-public class CliCommandTestBase extends CacheTestCase {
+public class CliCommandTestBase extends JUnit4CacheTestCase {
 
   private static final long serialVersionUID = 1L;
 
@@ -68,10 +67,6 @@ public class CliCommandTestBase extends CacheTestCase {
 
   private String jmxHost;
 
-  public CliCommandTestBase(String name) {
-    super(name);
-  }
-
   @Override
   public final void preTearDownCacheTestCase() throws Exception {
     preTearDownCliCommandTestBase();
@@ -238,7 +233,7 @@ public class CliCommandTestBase extends CacheTestCase {
     CommandResult result = executeCommand(shell, command.toString());
 
     if (!shell.isConnectedAndReady()) {
-      throw new TestException(
+      throw new AssertionError(
           "Connect command failed to connect to manager " + endpoint + " result=" + commandResultToString(result));
     }
 
@@ -273,9 +268,9 @@ public class CliCommandTestBase extends CacheTestCase {
       info("Started testable shell: " + shell);
       return shell;
     } catch (ClassNotFoundException e) {
-      throw new TestException(getStackTrace(e));
+      throw new AssertionError(getStackTrace(e));
     } catch (IOException e) {
-      throw new TestException(getStackTrace(e));
+      throw new AssertionError(getStackTrace(e));
     }
   }
 
@@ -338,9 +333,9 @@ public class CliCommandTestBase extends CacheTestCase {
     try {
       info("Executing command " + command + " with command Mgr " + CommandManager.getInstance());
     } catch (ClassNotFoundException cnfex) {
-      throw new TestException(getStackTrace(cnfex));
+      throw new AssertionError(getStackTrace(cnfex));
     } catch (IOException ioex) {
-      throw new TestException(getStackTrace(ioex));
+      throw new AssertionError(getStackTrace(ioex));
     }
 
     shell.executeCommand(command);
@@ -540,18 +535,18 @@ public class CliCommandTestBase extends CacheTestCase {
   }
 
   protected void info(String string) {
-    LogWriterUtils.getLogWriter().info(string);
+    getLogWriter().info(string);
   }
 
   protected void debug(String string) {
-    LogWriterUtils.getLogWriter().fine(string);
+    getLogWriter().fine(string);
   }
 
   protected void error(String string) {
-    LogWriterUtils.getLogWriter().error(string);
+    getLogWriter().error(string);
   }
 
   protected void error(String string, Throwable e) {
-    LogWriterUtils.getLogWriter().error(string, e);
+    getLogWriter().error(string, e);
   }
 }


[28/32] incubator-geode git commit: GEODE-1226: Prompt for a password to sign and upload to maven

Posted by ji...@apache.org.
GEODE-1226: Prompt for a password to sign and upload to maven

The signArchives and uploadArchives tasks require passwords to work.
Adding a flag to request a password prompt to fill in a password.

./gradlew uploadArchives -Paskpass

This will pop up a swing dialog box asking for passwords to sign and
upload archives.

This is not on by default because we don't want developers to have to
enter a password for every build. The build target depends on
signArchives but normally will skip it if the password property is not
set.


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 3fd1eec9114b2c72bb620578e118bceba8ea89a7
Parents: ce889fe
Author: Dan Smith <up...@apache.org>
Authored: Thu Apr 14 10:48:31 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Thu Apr 14 16:07:41 2016 -0700

----------------------------------------------------------------------
 .../apache/geode/gradle/PasswordDialog.groovy   | 41 ++++++++++++++++++++
 gradle/publish.gradle                           | 35 +++++++++++++++++
 2 files changed, 76 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fd1eec9/buildSrc/src/main/groovy/org/apache/geode/gradle/PasswordDialog.groovy
----------------------------------------------------------------------
diff --git a/buildSrc/src/main/groovy/org/apache/geode/gradle/PasswordDialog.groovy b/buildSrc/src/main/groovy/org/apache/geode/gradle/PasswordDialog.groovy
new file mode 100644
index 0000000..bcd2243
--- /dev/null
+++ b/buildSrc/src/main/groovy/org/apache/geode/gradle/PasswordDialog.groovy
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+import groovy.swing.SwingBuilder
+class PasswordDialog {
+  static String askPassword(String prompt) {
+    def password = ''
+    new SwingBuilder().edt {
+    dialog(modal: true, 
+        title: 'Password',
+        alwaysOnTop: true, 
+        locationRelativeTo: null,
+        pack: true, 
+        show: true
+    ) {
+      vbox { 
+        label(text: prompt)
+        input = passwordField()
+        button(defaultButton: true, text: 'OK', actionPerformed: {
+          password = input.password.toString(); // Set pass variable to value of input field
+          dispose(); // Close dialog
+        })
+      }
+      }
+    }
+    return password
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fd1eec9/gradle/publish.gradle
----------------------------------------------------------------------
diff --git a/gradle/publish.gradle b/gradle/publish.gradle
index 02b0e3c..9c251c7 100644
--- a/gradle/publish.gradle
+++ b/gradle/publish.gradle
@@ -28,6 +28,7 @@ subprojects {
     repositoryUrl = 'https://repository.apache.org/service/local/staging/deploy/maven2'
     snapshotRepositoryUrl = 'https://repository.apache.org/content/repositories/snapshots'
   }
+
   
   modifyPom {
     withXml {
@@ -119,3 +120,37 @@ subprojects {
     }
   }
 }
+
+//Prompt the user for a password to sign archives or upload artifacts, if requested
+gradle.taskGraph.whenReady { taskGraph ->
+  if (project.hasProperty('askpass')) {
+    if(taskGraph.allTasks.any {it instanceof Sign}) {
+      if(!project.hasProperty('signing.keyId') || !project.hasProperty('signing.secretKeyRingFile')) {
+        println "You must configure your signing.keyId and signing.secretKeyRingFile"
+        println "in ~/gradle/.properties in order to sign jars\n"
+        println "See https://cwiki.apache.org/confluence/display/GEODE/Release+Steps"
+        throw new GradleException("Signing key/keyring is missing")
+      }
+
+      if(!project.hasProperty('signing.password')) {
+        def password = PasswordDialog.askPassword("Please enter your password to unlock your gpg keyring for signing artifacts")
+        
+        subprojects { ext."signing.password" = password }
+      }
+    }
+
+    if(taskGraph.allTasks.any {it.name == 'uploadArchives'}) {
+      if(!project.hasProperty('nexusUsername')) {
+        println "You must configure your nexusUsername in ~/gradle/.properties in order to uploadArchives\n"
+        println "See https://cwiki.apache.org/confluence/display/GEODE/Release+Steps"
+        throw new GradleException("nexusUsername is missing")
+      }
+
+      if(!project.hasProperty('nexusPassword')) {
+        def password = PasswordDialog.askPassword("Please enter your apache password to uploadArchives to nexus")
+        
+        subprojects { ext."nexusPassword" = password }
+      }
+    }
+  }
+}


[29/32] incubator-geode git commit: GEODE-1162: make CliCommandTestBase class abstract

Posted by ji...@apache.org.
GEODE-1162: make CliCommandTestBase class abstract


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 70c03a7baf8e71ad1ba7577637469478faad57e6
Parents: 3fd1eec
Author: Kirk Lund <kl...@apache.org>
Authored: Thu Apr 14 14:18:41 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Thu Apr 14 16:27:45 2016 -0700

----------------------------------------------------------------------
 .../management/internal/cli/commands/CliCommandTestBase.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/70c03a7b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
index 55f5556..6a761b4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
@@ -50,7 +50,7 @@ import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
 /**
  * Base class for all the CLI/gfsh command dunit tests.
  */
-public class CliCommandTestBase extends JUnit4CacheTestCase {
+public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
 
   private static final long serialVersionUID = 1L;
 


[21/32] incubator-geode git commit: GEODE-1162: convert all CLI command DUnit tests to JUnit 4

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/GemFireSecurityExceptionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/GemFireSecurityExceptionTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/GemFireSecurityExceptionTest.java
index 0c048d3..5aa01ff 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/GemFireSecurityExceptionTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/GemFireSecurityExceptionTest.java
@@ -23,7 +23,6 @@ import java.io.NotSerializableException;
 import java.io.Serializable;
 import javax.naming.NamingException;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
 import org.apache.commons.lang.SerializationUtils;
 import org.junit.Before;
 import org.junit.Rule;
@@ -31,10 +30,13 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
  * Unit tests for {@link GemFireSecurityException}.
  */
-@Category(UnitTest.class)
+@Category({ UnitTest.class, SecurityTest.class })
 public class GemFireSecurityExceptionTest {
 
   private String message;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java
index c5e0ba5..ec054d7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/NotAuthorizedExceptionTest.java
@@ -25,7 +25,6 @@ import java.io.Serializable;
 import java.security.Principal;
 import javax.naming.NamingException;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
 import org.apache.commons.lang.SerializationUtils;
 import org.junit.Before;
 import org.junit.Rule;
@@ -33,10 +32,13 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
 /**
  * Unit tests for {@link NotAuthorizedException}.
  */
-@Category(UnitTest.class)
+@Category({ UnitTest.class, SecurityTest.class })
 public class NotAuthorizedExceptionTest {
 
   private String message;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
index 1db599f..179f29d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
@@ -29,6 +29,10 @@ import static com.gemstone.gemfire.test.dunit.Wait.*;
 import java.util.Properties;
 import javax.net.ssl.SSLHandshakeException;
 
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
@@ -44,16 +48,14 @@ import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 
 /**
  * Tests peer to peer authentication in Gemfire
  * 
  * @since 5.5
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class P2PAuthenticationDUnitTest extends JUnit4DistributedTestCase {
 
   private static VM locatorVM = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
index 663a17d..10ac68a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
@@ -18,16 +18,20 @@
  */
 package com.gemstone.gemfire.security;
 
+import static com.gemstone.gemfire.cache30.ClientServerTestCase.*;
 import static com.gemstone.gemfire.distributed.internal.DistributionConfig.*;
 import static com.gemstone.gemfire.internal.AvailablePort.*;
-import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Assert.assertEquals;
+import static com.gemstone.gemfire.test.dunit.Assert.assertFalse;
+import static com.gemstone.gemfire.test.dunit.Assert.assertNotNull;
+import static com.gemstone.gemfire.test.dunit.Assert.assertNull;
+import static com.gemstone.gemfire.test.dunit.Assert.assertTrue;
+import static com.gemstone.gemfire.test.dunit.Assert.fail;
 import static com.gemstone.gemfire.test.dunit.DistributedTestUtils.*;
 import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
 import static com.gemstone.gemfire.test.dunit.NetworkUtils.*;
 import static com.gemstone.gemfire.test.dunit.Wait.*;
 
-import static com.gemstone.gemfire.cache30.ClientServerTestCase.configureConnectionPoolWithNameAndFactory;
-
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -84,6 +88,7 @@ import com.gemstone.gemfire.pdx.PdxSerializable;
 import com.gemstone.gemfire.pdx.PdxWriter;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 
 /**
  * Contains utility methods for setting up servers/clients for authentication
@@ -93,7 +98,7 @@ import com.gemstone.gemfire.test.dunit.WaitCriterion;
  */
 public final class SecurityTestUtils {
 
-  private final DistributedTestCase distributedTestCase = new DistributedTestCase(getClass().getSimpleName()) {}; // TODO: delete
+  private final JUnit4DistributedTestCase distributedTestCase = new JUnit4DistributedTestCase() {}; // TODO: delete
 
   protected static final int NO_EXCEPTION = 0;
   protected static final int AUTHREQ_EXCEPTION = 1;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/generator/AuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/AuthzCredentialGenerator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/AuthzCredentialGenerator.java
index f39fc84..a561de0 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/AuthzCredentialGenerator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/AuthzCredentialGenerator.java
@@ -16,13 +16,6 @@
  */
 package com.gemstone.gemfire.security.generator;
 
-import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.security.AccessControl;
-import com.gemstone.gemfire.security.templates.DummyAuthorization;
-import com.gemstone.gemfire.security.templates.XmlAuthorization;
-import org.apache.logging.log4j.Logger;
-
 import java.security.Principal;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -31,6 +24,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.security.AccessControl;
+import com.gemstone.gemfire.security.templates.DummyAuthorization;
+import com.gemstone.gemfire.security.templates.XmlAuthorization;
+
 /**
  * Encapsulates obtaining authorized and unauthorized credentials for a given
  * operation in a region. Implementations will be for different kinds of

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/generator/CredentialGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/CredentialGenerator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/CredentialGenerator.java
index aee7ebb..fd079a2 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/CredentialGenerator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/CredentialGenerator.java
@@ -24,13 +24,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.logging.log4j.Logger;
+
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.security.AuthInitialize;
 import com.gemstone.gemfire.security.Authenticator;
 import com.gemstone.gemfire.security.templates.DummyAuthenticator;
 import com.gemstone.gemfire.security.templates.LdapUserAuthenticator;
 import com.gemstone.gemfire.security.templates.PKCSAuthenticator;
-import org.apache.logging.log4j.Logger;
 
 /**
  * Encapsulates obtaining valid and invalid credentials. Implementations will be

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/generator/DummyCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/DummyCredentialGenerator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/DummyCredentialGenerator.java
index b709dbc..77ffbd9 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/DummyCredentialGenerator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/DummyCredentialGenerator.java
@@ -16,12 +16,12 @@
  */
 package com.gemstone.gemfire.security.generator;
 
-import com.gemstone.gemfire.security.templates.DummyAuthenticator;
-import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
-
 import java.security.Principal;
 import java.util.Properties;
 
+import com.gemstone.gemfire.security.templates.DummyAuthenticator;
+import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
+
 public class DummyCredentialGenerator extends CredentialGenerator {
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java
index bbd9528..7bdcd9a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/LdapUserCredentialGenerator.java
@@ -16,18 +16,19 @@
  */
 package com.gemstone.gemfire.security.generator;
 
+import java.security.Principal;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.logging.log4j.Logger;
+
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.test.dunit.Assert;
-import com.gemstone.gemfire.util.test.TestUtil;
 import com.gemstone.gemfire.security.templates.LdapUserAuthenticator;
 import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
-import org.apache.logging.log4j.Logger;
-
-import java.security.Principal;
-import java.util.Properties;
-import java.util.Random;
+import com.gemstone.gemfire.test.dunit.Assert;
+import com.gemstone.gemfire.util.test.TestUtil;
 
 public class LdapUserCredentialGenerator extends CredentialGenerator {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/generator/PKCSCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/PKCSCredentialGenerator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/PKCSCredentialGenerator.java
index 6d33493..09381b8 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/PKCSCredentialGenerator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/PKCSCredentialGenerator.java
@@ -16,15 +16,15 @@
  */
 package com.gemstone.gemfire.security.generator;
 
-import com.gemstone.gemfire.util.test.TestUtil;
-import com.gemstone.gemfire.security.templates.PKCSAuthInit;
-import com.gemstone.gemfire.security.templates.PKCSAuthenticator;
-
 import java.security.Principal;
 import java.security.Provider;
 import java.security.Security;
 import java.util.Properties;
 
+import com.gemstone.gemfire.security.templates.PKCSAuthInit;
+import com.gemstone.gemfire.security.templates.PKCSAuthenticator;
+import com.gemstone.gemfire.util.test.TestUtil;
+
 public class PKCSCredentialGenerator extends CredentialGenerator {
 
   public static String keyStoreDir = getKeyStoreDir();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java
index ff23f78..6f7cf29 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/SSLCredentialGenerator.java
@@ -16,15 +16,16 @@
  */
 package com.gemstone.gemfire.security.generator;
 
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.security.AuthenticationFailedException;
-import org.apache.logging.log4j.Logger;
-
 import java.io.File;
 import java.io.IOException;
 import java.security.Principal;
 import java.util.Properties;
 
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.security.AuthenticationFailedException;
+
 public class SSLCredentialGenerator extends CredentialGenerator {
 
   private static final Logger logger = LogService.getLogger();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java
index b29f16b..6d0c583 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/UserPasswordWithExtraPropsAuthInit.java
@@ -16,14 +16,14 @@
  */
 package com.gemstone.gemfire.security.generator;
 
+import java.util.Iterator;
+import java.util.Properties;
+
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.security.AuthInitialize;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 
-import java.util.Iterator;
-import java.util.Properties;
-
 /**
  * An {@link AuthInitialize} implementation that obtains the user name and
  * password as the credentials from the given set of properties. If 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/generator/XmlAuthzCredentialGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/XmlAuthzCredentialGenerator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/XmlAuthzCredentialGenerator.java
index 5d07004..7dcf5ae 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/generator/XmlAuthzCredentialGenerator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/generator/XmlAuthzCredentialGenerator.java
@@ -16,16 +16,16 @@
  */
 package com.gemstone.gemfire.security.generator;
 
-import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import com.gemstone.gemfire.util.test.TestUtil;
-import com.gemstone.gemfire.security.templates.UsernamePrincipal;
-import com.gemstone.gemfire.security.templates.XmlAuthorization;
-
 import java.security.Principal;
 import java.util.HashSet;
 import java.util.Properties;
 import java.util.Set;
 
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.security.templates.UsernamePrincipal;
+import com.gemstone.gemfire.security.templates.XmlAuthorization;
+import com.gemstone.gemfire.util.test.TestUtil;
+
 public class XmlAuthzCredentialGenerator extends AuthzCredentialGenerator {
 
   private static final String dummyXml = "authz-dummy.xml";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/templates/DummyAuthorization.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/DummyAuthorization.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/DummyAuthorization.java
index a76a46f..17bc5e1 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/DummyAuthorization.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/DummyAuthorization.java
@@ -16,6 +16,10 @@
  */
 package com.gemstone.gemfire.security.templates;
 
+import java.security.Principal;
+import java.util.HashSet;
+import java.util.Set;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.operations.OperationContext;
@@ -24,10 +28,6 @@ import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.security.AccessControl;
 import com.gemstone.gemfire.security.NotAuthorizedException;
 
-import java.security.Principal;
-import java.util.HashSet;
-import java.util.Set;
-
 /**
  * A dummy implementation of the {@code AccessControl} interface that
  * allows authorization depending on the format of the {@code Principal}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/templates/LdapUserAuthenticator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/LdapUserAuthenticator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/LdapUserAuthenticator.java
index fd2286e..b92a08a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/LdapUserAuthenticator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/LdapUserAuthenticator.java
@@ -22,12 +22,13 @@ import javax.naming.Context;
 import javax.naming.directory.DirContext;
 import javax.naming.directory.InitialDirContext;
 
+import org.apache.logging.log4j.Logger;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.Authenticator;
-import org.apache.logging.log4j.Logger;
 
 /**
  * An implementation of {@link Authenticator} that uses LDAP.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java
index 0b38a7b..9f1ccec 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java
@@ -24,12 +24,13 @@ import java.security.Signature;
 import java.security.cert.X509Certificate;
 import java.util.Properties;
 
+import org.apache.logging.log4j.Logger;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.security.AuthInitialize;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
-import org.apache.logging.log4j.Logger;
 
 /**
  * An {@link AuthInitialize} implementation that obtains the digital signature

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthenticator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthenticator.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthenticator.java
index 971cf60..ac5939d 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthenticator.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthenticator.java
@@ -29,12 +29,13 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.logging.log4j.Logger;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 import com.gemstone.gemfire.security.Authenticator;
-import org.apache.logging.log4j.Logger;
 
 /**
  * An implementation of {@link Authenticator} that uses PKCS.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java
index e0bc1e4..677e2d4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSPrincipalTest.java
@@ -16,14 +16,15 @@
  */
 package com.gemstone.gemfire.security.templates;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import static org.assertj.core.api.Assertions.*;
+
+import java.io.Serializable;
+
 import org.apache.commons.lang.SerializationUtils;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.Serializable;
-
-import static org.assertj.core.api.Assertions.assertThat;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Unit tests for {@link PKCSPrincipal}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java
index 34161ac..6c05090 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java
@@ -16,13 +16,13 @@
  */
 package com.gemstone.gemfire.security.templates;
 
+import java.util.Properties;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.security.AuthInitialize;
 import com.gemstone.gemfire.security.AuthenticationFailedException;
 
-import java.util.Properties;
-
 /**
  * An {@link AuthInitialize} implementation that obtains the user name and
  * password as the credentials from the given set of properties.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java
index e762d06..bce19b4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UsernamePrincipalTest.java
@@ -16,14 +16,15 @@
  */
 package com.gemstone.gemfire.security.templates;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import static org.assertj.core.api.Assertions.*;
+
+import java.io.Serializable;
+
 import org.apache.commons.lang.SerializationUtils;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.Serializable;
-
-import static org.assertj.core.api.Assertions.assertThat;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
  * Unit tests for {@link UsernamePrincipal}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlAuthorization.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlAuthorization.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlAuthorization.java
index b8f2e50..2153fb4 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlAuthorization.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlAuthorization.java
@@ -29,15 +29,6 @@ import java.util.regex.Pattern;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
-import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.operations.ExecuteFunctionOperationContext;
-import com.gemstone.gemfire.cache.operations.OperationContext;
-import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import com.gemstone.gemfire.cache.operations.QueryOperationContext;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.security.AccessControl;
-import com.gemstone.gemfire.security.NotAuthorizedException;
 import org.w3c.dom.Attr;
 import org.w3c.dom.Document;
 import org.w3c.dom.NamedNodeMap;
@@ -48,6 +39,16 @@ import org.xml.sax.InputSource;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXParseException;
 
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.operations.ExecuteFunctionOperationContext;
+import com.gemstone.gemfire.cache.operations.OperationContext;
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.cache.operations.QueryOperationContext;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.security.AccessControl;
+import com.gemstone.gemfire.security.NotAuthorizedException;
+
 /**
  * An implementation of the {@link AccessControl} interface that allows
  * authorization using the permissions as specified in the given XML

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlErrorHandler.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlErrorHandler.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlErrorHandler.java
index 19d10c6..e2b286b 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlErrorHandler.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/XmlErrorHandler.java
@@ -16,13 +16,14 @@
  */
 package com.gemstone.gemfire.security.templates;
 
-import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.internal.logging.LogService;
 import org.apache.logging.log4j.Logger;
 import org.xml.sax.ErrorHandler;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXParseException;
 
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.internal.logging.LogService;
+
 /**
  * Implementation of {@link ErrorHandler} interface to handle validation errors
  * while XML parsing.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/AsyncInvocation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/AsyncInvocation.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/AsyncInvocation.java
index a612f87..a9427d3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/AsyncInvocation.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/AsyncInvocation.java
@@ -16,21 +16,25 @@
  */
 package com.gemstone.gemfire.test.dunit;
 
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicReference;
 
-import com.gemstone.gemfire.InternalGemFireError;
 import com.gemstone.gemfire.SystemFailure;
 
 /**
- * <P>An <code>AsyncInvocation</code> represents the invocation of a
- * remote invocation that executes asynchronously from its caller.  An
- * instanceof <code>AsyncInvocation</code> provides information about
- * the invocation such as any exception that it may have thrown.</P>
+ * An {@code AsyncInvocation} represents the invocation of a remote invocation
+ * that executes asynchronously from its caller.  An instance of
+ * {@code AsyncInvocation} provides information about the invocation such as
+ * any exception that it may have thrown.
  *
- * <P>Because it is a <code>Thread</code>, an
- * <code>AsyncInvocation</code> can be used as follows:</P>
+ * <p>{@code AsyncInvocation} can be used as follows:
  *
- * <PRE>
+ * <pre>
  *   AsyncInvocation ai1 = vm.invokeAsync(() -> Test.method1());
  *   AsyncInvocation ai2 = vm.invokeAsync(() -> Test.method2());
  *
@@ -42,87 +46,65 @@ import com.gemstone.gemfire.SystemFailure;
  *   if (ai2.exceptionOccurred()) {
  *     throw ai2.getException();
  *   }
- * </PRE>
+ * </pre>
  *
+ * @param <V> The result type returned by this AsyncInvocation's {@code get} methods
  * @see VM#invokeAsync(Class, String)
  */
-public class AsyncInvocation<T> extends Thread {
-  //@todo davidw Add the ability to get a return value back from the
-  //async method call.  (Use a static ThreadLocal field that is
-  //accessible from the Runnable used in VM#invoke)
-  
-  private static final ThreadLocal returnValue = new ThreadLocal();
+public class AsyncInvocation<V> implements Future<V> {
+  // TODO:davidw: Add the ability to get a return value back from the
+  // async method call.  (Use a static ThreadLocal field that is
+  // accessible from the Runnable used in VM#invoke)
+  // TODO:?: reimplement using Futures
+
+  private static final long DEFAULT_JOIN_MILLIS = 60 * 1000;
 
-  /** The singleton the thread group */
-  private static final ThreadGroup GROUP = new AsyncInvocationGroup();
+  private final Thread thread;
 
-  ///////////////////// Instance Fields  /////////////////////
+  private final AtomicReference<V> resultValue = new AtomicReference<>();
 
-  /** An exception thrown while this async invocation ran */
-  protected volatile Throwable exception;
+  /** An exception thrown while this {@code AsyncInvocation} ran */
+  private final AtomicReference<Throwable> resultThrowable = new AtomicReference<>();
 
-  /** The object (or class) that is the receiver of this asyn method
-   * invocation */
-  private Object receiver;
+  /** The object (or class) that is the target of this {@code AsyncInvocation} */
+  private Object target;
 
   /** The name of the method being invoked */
   private String methodName;
-  
-  /** The returned object if any */
-  public volatile T returnedObj = null;
-
-  //////////////////////  Constructors  //////////////////////
 
+  /** True if this {@code AsyncInvocation} has been cancelled */
+  private boolean cancelled;
+  
   /**
-   * Creates a new <code>AsyncInvocation</code>
+   * Creates a new {@code AsyncInvocation}.
    *
-   * @param receiver
-   *        The object or {@link Class} on which the remote method was
-   *        invoked
-   * @param methodName
-   *        The name of the method being invoked
-   * @param work
-   *        The actual invocation of the method
+   * @param  target
+   *         The object or {@link Class} on which the remote method was
+   *         invoked
+   * @param  methodName
+   *         The name of the method being invoked
+   * @param  work
+   *         The actual invocation of the method
    */
-  public AsyncInvocation(Object receiver, String methodName, Runnable work) {
-    super(GROUP, work, getName(receiver, methodName));
-    this.receiver = receiver;
+  public AsyncInvocation(final Object target, final String methodName, final Callable<V> work) {
+    this.target = target;
     this.methodName = methodName;
-    this.exception = null;
+    this.thread = new Thread(new AsyncInvocationGroup(), runnable(work), getName(target, methodName));
   }
 
-  //////////////////////  Static Methods  /////////////////////
-
   /**
-   * Returns the name of a <code>AsyncInvocation</code> based on its
-   * receiver and method name.
-   */
-  private static String getName(Object receiver, String methodName) {
-    StringBuffer sb = new StringBuffer(methodName);
-    sb.append(" invoked on ");
-    if (receiver instanceof Class) {
-      sb.append("class ");
-      sb.append(((Class) receiver).getName());
-
-    } else {
-      sb.append("an instance of ");
-      sb.append(receiver.getClass().getName());
-    }
-
-    return sb.toString();
-  }
-
-  /////////////////////  Instance Methods  ////////////////////
-
-  /**
-   * Returns the receiver of this async method invocation
+   * Returns the target of this async method invocation.
+   *
+   * @deprecated This method is not required for anything.
    */
-  public Object getReceiver() {
-    return this.receiver;
+  public Object getTarget() {
+    return this.target;
   }
 
   /**
-   * Returns the name of the method being invoked remotely
+   * Returns the name of the method being invoked remotely.
+   *
+   * @deprecated This method is not required for anything.
    */
   public String getMethodName() {
     return this.methodName;
@@ -131,86 +113,383 @@ public class AsyncInvocation<T> extends Thread {
   /**
    * Returns whether or not an exception occurred during this async
    * method invocation.
+   *
+   * @throws AssertionError if this {@code AsyncInvocation} is not done.
    */
   public boolean exceptionOccurred() {
-    if (this.isAlive()) {
-      throw new InternalGemFireError("Exception status not available while thread is alive.");
-    }
-    return this.exception != null;
+    return getException() != null;
   }
 
   /**
    * Returns the exception that was thrown during this async method
    * invocation.
+   *
+   * @throws AssertionError if this {@code AsyncInvocation} is not done.
    */
   public Throwable getException() {
-    if (this.isAlive()) {
-      throw new InternalGemFireError("Exception status not available while thread is alive.");
+    try {
+      checkIsDone("Exception status not available while thread is alive.");
+    } catch (IllegalStateException illegalStateException) {
+      throw new AssertionError(illegalStateException);
     }
-    if (this.exception instanceof RMIException) {
-      return ((RMIException) this.exception).getCause();
+
+    if (this.resultThrowable.get() instanceof RMIException) { // TODO:klund: delete our RMIException
+      return this.resultThrowable.get().getCause();
 
     } else {
-      return this.exception;
+      return this.resultThrowable.get();
     }
   }
 
-  //////////////////////  Inner Classes  //////////////////////
+  /**
+   * Throws {@code AssertionError} wrapping any {@code Exception} thrown by
+   * this {@code AsyncInvocation}.
+   *
+   * @return this {@code AsyncInvocation}
+   *
+   * @throws AssertionError wrapping any {@code Exception} thrown by this
+   *         {@code AsyncInvocation}.
+   */
+  public AsyncInvocation<V> checkException() {
+    if (this.resultThrowable.get() != null) {
+      throw new AssertionError("An exception occurred during asynchronous invocation.", getException());
+    }
+    return this;
+  }
 
   /**
-   * A <code>ThreadGroup</code> that notices when an exception occurs
-   * during an <code>AsyncInvocation</code>.
-   * 
-   * TODO: reimplement using Futures
+   * Returns the result of this {@code AsyncInvocation}.
+   *
+   * @return the result of this {@code AsyncInvocation}
+   *
+   * @throws AssertionError wrapping any {@code Exception} thrown by this
+   *         {@code AsyncInvocation}.
+   *
+   * @throws AssertionError wrapping a {@code TimeoutException} if this
+   *         {@code AsyncInvocation} fails to complete within the default
+   *         timeout of 60 seconds as defined by {@link #DEFAULT_JOIN_MILLIS}.
+   *
+   * @throws InterruptedException if the current thread is interrupted.
+   *
+   * @deprecated Please use {@link #get()} instead.
    */
-  private static class AsyncInvocationGroup extends ThreadGroup {
-    AsyncInvocationGroup() {
-      super("Async Invocations");
+  public V getResult() throws InterruptedException {
+    join();
+    checkException();
+    checkIsDone("Return value not available while thread is alive.");
+    return this.resultValue.get();
+  }
+
+  /**
+   * Returns the result of this {@code AsyncInvocation}.
+   *
+   * @param  millis
+   *         the time to wait in milliseconds
+   *
+   * @return the result of this {@code AsyncInvocation}
+   *
+   * @throws AssertionError wrapping any {@code Exception} thrown by this
+   *         {@code AsyncInvocation}.
+   *
+   * @throws AssertionError wrapping a {@code TimeoutException} if this
+   *         {@code AsyncInvocation} fails to complete within the specified
+   *         timeout of {@code millis}.
+   *
+   * @throws InterruptedException if the current thread is interrupted.
+   *
+   * @deprecated Please use {@link #get(long, TimeUnit)} instead.
+   */
+  public V getResult(final long millis) throws InterruptedException {
+    try {
+      return get(millis, TimeUnit.MILLISECONDS);
+    } catch (ExecutionException executionException) {
+      throw new AssertionError(executionException);
+    } catch (TimeoutException timeoutException) {
+      throw new AssertionError(timeoutException);
     }
+  }
 
-    public void uncaughtException(Thread thread, Throwable throwable) {
-      if (throwable instanceof VirtualMachineError) {
-        SystemFailure.setFailure((VirtualMachineError)throwable); // don't throw
-      }
-      if (thread instanceof AsyncInvocation) {
-        ((AsyncInvocation) thread).exception = throwable;
+  /**
+   * Returns the result of this {@code AsyncInvocation}.
+   *
+   * @return the result of this {@code AsyncInvocation}
+   *
+   * @throws AssertionError if this {@code AsyncInvocation} is not done.
+   *
+   * @deprecated Please use {@link #get()} instead.
+   */
+  public V getReturnValue() {
+    checkIsDone("Return value not available while thread is alive.");
+    return this.resultValue.get();
+  }
+
+  /**
+   * Waits at most {@code millis} milliseconds for this
+   * {@code AsyncInvocation} to complete. A timeout of {@code 0} means to wait
+   * forever.
+   *
+   * @param  millis
+   *         the time to wait in milliseconds
+   *
+   * @return this {@code AsyncInvocation}
+   *
+   * @throws IllegalArgumentException if the value of {@code millis} is
+   *         negative.
+   *
+   * @throws InterruptedException if the current thread is interrupted.
+   */
+  public synchronized AsyncInvocation<V> join(final long millis) throws InterruptedException {
+    this.thread.join(millis);
+    return this;
+  }
+
+  /**
+   * Waits at most {@code millis} milliseconds plus {@code nanos} nanoseconds
+   * for this {@code AsyncInvocation} to complete.
+   *
+   * @param  millis
+   *         the time to wait in milliseconds
+   * @param  nanos
+   *         {@code 0-999999} additional nanoseconds to wait
+   *
+   * @return this {@code AsyncInvocation}
+   *
+   * @throws IllegalArgumentException
+   *         if the value of {@code millis} is negative, or the value
+   *         of {@code nanos} is not in the range {@code 0-999999}.
+   *
+   * @throws InterruptedException if the current thread is interrupted.
+   */
+  public synchronized AsyncInvocation<V> join(final long millis, final int nanos) throws InterruptedException {
+    this.thread.join(millis, nanos);
+    return this;
+  }
+
+  /**
+   * Waits for this thread to die up to a default of 60 seconds as defined by
+   * {@link #DEFAULT_JOIN_MILLIS}.
+   *
+   * @return this {@code AsyncInvocation}
+   *
+   * @throws InterruptedException if the current thread is interrupted.
+   */
+  public AsyncInvocation<V> join() throws InterruptedException {
+    // do NOT invoke Thread#join() without a timeout
+    join(DEFAULT_JOIN_MILLIS);
+    return this;
+  }
+
+  /**
+   * Start this {@code AsyncInvocation}.
+   *
+   * @return this {@code AsyncInvocation}
+   */
+  public synchronized AsyncInvocation<V> start() {
+    this.thread.start();
+    return this;
+  }
+
+  /**
+   * Return this {@code AsyncInvocation}'s work thread.
+   *
+   * @return this {@code AsyncInvocation}'s work thread.
+   */
+  public synchronized Thread getThread() {
+    return this.thread;
+  }
+
+  /**
+   * Tests if this {@code AsyncInvocation}'s thread is alive. A thread is alive
+   * if it has been started and has not yet died.
+   *
+   * @return {@code true} if this {@code AsyncInvocation}'s thread is alive;
+   *         {@code false} otherwise.
+   */
+  public synchronized boolean isAlive() {
+    return this.thread.isAlive();
+  }
+
+  @Override
+  public synchronized boolean isCancelled() {
+    return this.cancelled;
+  }
+
+  @Override
+  public synchronized boolean isDone() {
+    return !this.thread.isAlive(); //state != NEW;
+  }
+
+  @Override
+  public synchronized boolean cancel(final boolean mayInterruptIfRunning) {
+    if (this.thread.isAlive()) {
+      if (mayInterruptIfRunning) {
+        this.cancelled = true;
+        this.thread.interrupt();
+        return true;
       }
     }
+    return false;
   }
-  
-  public T getResult() throws Throwable {
-    join();
-    if(this.exceptionOccurred()) {
-      throw new Exception("An exception occured during async invocation", this.exception);
+
+  /**
+   * Waits if necessary for the work to complete, and then returns the result
+   * of this {@code AsyncInvocation}.
+   *
+   * @return the result of this {@code AsyncInvocation}
+   *
+   * @throws AssertionError wrapping any {@code Exception} thrown by this
+   *         {@code AsyncInvocation}.
+   *
+   * @throws AssertionError wrapping a {@code TimeoutException} if this
+   *         {@code AsyncInvocation} fails to complete within the default
+   *         timeout of 60 seconds as defined by {@link #DEFAULT_JOIN_MILLIS}.
+   *
+   * @throws CancellationException if the computation was cancelled
+   *
+   * @throws ExecutionException if the computation threw an exception
+   *
+   * @throws InterruptedException if the current thread is interrupted.
+   */
+  @Override
+  public V get() throws ExecutionException, InterruptedException {
+    try {
+      return get(DEFAULT_JOIN_MILLIS, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException timeoutException) {
+      throw new AssertionError(timeoutException);
     }
-    return this.returnedObj;
   }
-  
-  public T getResult(long waitTime) throws Throwable {
-    join(waitTime);
-    if(this.isAlive()) {
-      throw new TimeoutException();
-    }
-    if(this.exceptionOccurred()) {
-      throw new Exception("An exception occured during async invocation", this.exception);
+
+  /**
+   * Waits if necessary for at most the given time for the computation
+   * to complete, and then retrieves its result, if available.
+   *
+   * @param  timeout the maximum time to wait
+   * @param  unit the time unit of the timeout argument
+   *
+   * @return the result of this {@code AsyncInvocation}
+   *
+   * @throws AssertionError wrapping any {@code Exception} thrown by this
+   *         {@code AsyncInvocation}.
+   *
+   * @throws AssertionError wrapping a {@code TimeoutException} if this
+   *         {@code AsyncInvocation} fails to complete within the default
+   *         timeout of 60 seconds as defined by {@link #DEFAULT_JOIN_MILLIS}.
+   *
+   * @throws CancellationException if the computation was cancelled
+   *
+   * @throws ExecutionException if the computation threw an exception
+   *
+   * @throws InterruptedException if the current thread is interrupted.
+   *
+   * @throws TimeoutException if the wait timed out
+   */
+  @Override
+  public V get(final long timeout, final TimeUnit unit) throws ExecutionException, InterruptedException, TimeoutException {
+    long millis = unit.toMillis(timeout);
+    join(millis);
+    timeoutIfAlive(millis);
+    checkException();
+    return this.resultValue.get();
+  }
+
+  /**
+   * Returns the identifier of this {@code AsyncInvocation}'s thread. The
+   * thread ID is a positive <tt>long</tt> number generated when this thread
+   * was created. The thread ID is unique and remains unchanged during its
+   * lifetime. When a thread is terminated, this thread ID may be reused.
+   *
+   * @return this {@code AsyncInvocation}'s thread's ID.
+   */
+  public long getId() {
+    return this.thread.getId();
+  }
+
+  @Override
+  public String toString() {
+    return "AsyncInvocation{" + "target=" + target + ", methodName='" + methodName + '\'' + '}';
+  }
+
+  /**
+   * Throws {@code IllegalStateException} if this {@code AsyncInvocation} is
+   * not done.
+   *
+   * @param  message
+   *         The value to be used in constructing detail message
+   *
+   * @return this {@code AsyncInvocation}
+   *
+   * @throws IllegalStateException if this {@code AsyncInvocation} is not done.
+   */
+  private AsyncInvocation<V> checkIsDone(final String message) {
+    if (this.thread.isAlive()) {
+      throw new IllegalStateException(message);
     }
-    return this.returnedObj;
+    return this;
   }
 
-  public T getReturnValue() {
-    if (this.isAlive()) {
-      throw new InternalGemFireError("Return value not available while thread is alive.");
+  /**
+   * Throws {@code AssertionError} wrapping a {@code TimeoutException} if this
+   * {@code AsyncInvocation} fails to complete within the default timeout of 60
+   * seconds as defined by {@link #DEFAULT_JOIN_MILLIS}.
+   *
+   * @return this {@code AsyncInvocation}
+   *
+   * @throws TimeoutException if this {@code AsyncInvocation} fails to complete
+   *         within the default timeout of 60 seconds as defined by
+   *         {@link #DEFAULT_JOIN_MILLIS}.
+   */
+  private AsyncInvocation<V> timeoutIfAlive(final long timeout) throws TimeoutException {
+    if (this.thread.isAlive()) {
+      throw new TimeoutException("Timed out waiting " + timeout + " milliseconds for AsyncInvocation to complete.");
     }
-    return this.returnedObj;
+    return this;
   }
-  
-  public void run() {
-    super.run();
-    this.returnedObj = (T) returnValue.get();
-    returnValue.set(null);
+
+  private Runnable runnable(final Callable<V> work) {
+    return () -> {
+        try {
+          resultValue.set(work.call());
+        } catch (Throwable throwable) {
+          resultThrowable.set(throwable);
+        }
+    };
+  }
+
+  /**
+   * Returns the name of a {@code AsyncInvocation} based on its
+   * {@code targetObject} and {@code methodName}.
+   */
+  private static String getName(final Object target, final String methodName) {
+    StringBuilder sb = new StringBuilder(methodName);
+    sb.append(" invoked on ");
+    if (target instanceof Class) {
+      sb.append("class ");
+      sb.append(((Class) target).getName());
+
+    } else {
+      sb.append("an instance of ");
+      sb.append(target.getClass().getName());
+    }
+
+    return sb.toString();
   }
 
-  static void setReturnValue(Object v) {
-    returnValue.set(v);
+  /**
+   * A {@code ThreadGroup} that notices when an exception occurs
+   * during an {@code AsyncInvocation}.
+   */
+  private class AsyncInvocationGroup extends ThreadGroup {
+
+    private AsyncInvocationGroup() {
+      super("Async Invocations");
+    }
+
+    @Override
+    public void uncaughtException(Thread thread, Throwable throwable) {
+      if (throwable instanceof VirtualMachineError) {
+        SystemFailure.setFailure((VirtualMachineError)throwable); // don't throw
+      }
+      resultThrowable.set(throwable);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
index c0245f2..b7d33b0 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestCase.java
@@ -16,24 +16,22 @@
  */
 package com.gemstone.gemfire.test.dunit;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.test.dunit.internal.JUnit3DistributedTestCase;
-import com.gemstone.gemfire.test.dunit.standalone.DUnitLauncher;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.experimental.categories.Category;
 
 /**
  * This class is the superclass of all distributed unit tests.
- *
  */
 @Category(DistributedTest.class)
 @SuppressWarnings("serial")
 public abstract class DistributedTestCase extends JUnit3DistributedTestCase {
   
   /**
-   * Creates a new <code>DistributedTestCase</code> test with the given name.
+   * Creates a new {@code DistributedTestCase} test with the given name.
    */
   public DistributedTestCase(final String name) {
     super(name);
-    DUnitLauncher.launchIfNeeded();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java
index c1850de..9f69dab 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/DistributedTestUtils.java
@@ -16,7 +16,7 @@
  */
 package com.gemstone.gemfire.test.dunit;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
 import java.io.File;
 import java.util.Iterator;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/ThreadUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/ThreadUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/ThreadUtils.java
index fe8f260..6ac6449 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/ThreadUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/ThreadUtils.java
@@ -16,8 +16,8 @@
  */
 package com.gemstone.gemfire.test.dunit;
 
-import static org.junit.Assert.fail;
 import static com.gemstone.gemfire.test.dunit.Jitter.*;
+import static org.junit.Assert.*;
 
 import org.apache.logging.log4j.Logger;
 
@@ -108,6 +108,17 @@ public class ThreadUtils {
 
   /**
    * Wait for a thread to join.
+   *
+   * @param async async invocation to wait on
+   * @param timeoutMilliseconds maximum time to wait
+   * @throws AssertionError if the thread does not terminate
+   */
+  public static void join(final AsyncInvocation<?> async, final long timeoutMilliseconds) {
+    join(async.getThread(), timeoutMilliseconds);
+  }
+
+  /**
+   * Wait for a thread to join.
    * 
    * @param thread thread to wait on
    * @param timeoutMilliseconds maximum time to wait

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java
index 962978f..eb52ffc 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/VM.java
@@ -21,15 +21,18 @@ import java.io.PrintWriter;
 import java.io.Serializable;
 import java.io.StringWriter;
 import java.rmi.RemoteException;
+import java.util.concurrent.Callable;
 
+import com.jayway.awaitility.Awaitility;
+import hydra.MethExecutorResult;
+
+import com.gemstone.gemfire.internal.process.ProcessUtils;
 import com.gemstone.gemfire.test.dunit.standalone.BounceResult;
 import com.gemstone.gemfire.test.dunit.standalone.RemoteDUnitVMIF;
-
-import hydra.MethExecutorResult;
+import com.gemstone.gemfire.test.dunit.standalone.StandAloneDUnitEnv;
 
 /**
  * This class represents a Java Virtual Machine that runs on a host.
- *
  */
 @SuppressWarnings("serial")
 public class VM implements Serializable {
@@ -46,11 +49,33 @@ public class VM implements Serializable {
   /** The state of this VM */
   private volatile boolean available;
 
-  ////////////////////  Constructors  ////////////////////
+  /**
+   * Returns the {@code VM} identity. For {@link StandAloneDUnitEnv} the number
+   * returned is a zero-based sequence representing the order in with
+   * the DUnit {@code VM}s were launched.
+   */
+  public static int getCurrentVMNum() {
+    return DUnitEnv.get().getVMID();
+  }
+
+  /**
+   * Returns the total number of {@code VM}s on all {@code Host}s (note that
+   * DUnit currently only supports one {@code Host}).
+   */
+  public static int getVMCount() {
+    int count = 0;
+    for (int h = 0; h < Host.getHostCount(); h++) {
+      Host host = Host.getHost(h);
+      count += host.getVMCount();
+    }
+    return count;
+  }
 
   /**
-   * Creates a new <code>VM</code> that runs on a given host with a
-   * given process id.
+   * Creates a new {@code VM} that runs on a given host with a given process 
+   * id.
+   * 
+   * TODO: change pid to reflect value from {@link ProcessUtils#identifyPid()}
    */
   public VM(final Host host, final int pid, final RemoteDUnitVMIF client) {
     this.host = host;
@@ -59,452 +84,378 @@ public class VM implements Serializable {
     this.available = true;
   }
 
-  //////////////////////  Accessors  //////////////////////
-
   /**
-   * Returns the host on which this <code>VM</code> runs
+   * Returns the {@code Host} on which this {@code VM} is running.
    */
   public Host getHost() {
     return this.host;
   }
 
   /**
-   * Returns the process id of this <code>VM</code>
+   * Returns the process id of this {@code VM}.
    */
   public int getPid() {
     return this.pid;
   }
 
-  /////////////////  Remote Method Invocation  ///////////////
-
   /**
-   * Invokes a static zero-arg method  with an {@link Object} or
-   * <code>void</code> return type in this VM.  If the return type of
-   * the method is <code>void</code>, <code>null</code> is returned.
-   *
-   * @param targetClass
-   *        The class on which to invoke the method
-   * @param methodName
-   *        The name of the method to invoke
+   * Invokes a static zero-arg method  with an {@link Object} or {@code void} 
+   * return type in this {@code VM}.  If the return type of the method is
+   * {@code void}, {@code null} is returned.
    *
+   * @param  targetClass
+   *         The class on which to invoke the method
+   * @param  methodName
+   *         The name of the method to invoke
+   *         
    * @throws RMIException
-   *         An exception occurred on while invoking the method in
+   *         Wraps any underlying exception thrown while invoking the method in
    *         this VM
-   * @deprecated Use {@link #invoke(SerializableCallableIF)} instead        
+   *         
+   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead        
    */
   public Object invoke(final Class targetClass, final String methodName) {
     return invoke(targetClass, methodName, new Object[0]);
   }
 
   /**
-   * Asynchronously invokes a static zero-arg method with an {@link
-   * Object} or <code>void</code> return type in this VM.  If the
-   * return type of the method is <code>void</code>, <code>null</code>
-   * is returned.
+   * Asynchronously invokes a static zero-arg method with an {@code Object} or
+   * {@code void} return type in this VM.  If the return type of the method is
+   * {@code void}, {@code null} is returned.
    *
-   * @param targetClass
-   *        The class on which to invoke the method
-   * @param methodName
-   *        The name of the method to invoke
-   * @deprecated Use {@link #invoke(SerializableCallableIF)} instead       
+   * @param  targetClass
+   *         The class on which to invoke the method
+   * @param  methodName
+   *         The name of the method to invoke
+   *         
+   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead       
    */
   public AsyncInvocation invokeAsync(final Class targetClass, final String methodName) {
     return invokeAsync(targetClass, methodName, null);
   }
 
   /**
-   * Invokes a static method with an {@link Object} or
-   * <code>void</code> return type in this VM.  If the return type of
-   * the method is <code>void</code>, <code>null</code> is returned.
-   *
-   * @param targetClass
-   *        The class on which to invoke the method
-   * @param methodName
-   *        The name of the method to invoke
-   * @param args
-   *        Arguments passed to the method call (must be {@link
-   *        java.io.Serializable}). 
+   * Invokes a static method with an {@link Object} or {@code void} return type
+   * in this VM.  If the return type of the method is {@code void},
+   * {@code null} is returned.
    *
+   * @param  targetClass
+   *         The class on which to invoke the method
+   * @param  methodName
+   *         The name of the method to invoke
+   * @param  args
+   *         Arguments passed to the method call (must be
+   *         {@link java.io.Serializable}).
+   *         
    * @throws RMIException
-   *         An exception occurred on while invoking the method in
-   *         this VM
-   * @deprecated Use {@link #invoke(SerializableCallableIF)} instead        
+   *         Wraps any underlying exception thrown while invoking the method in
+   *         this {@code VM}
+   *         
+   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead        
    */
-  public Object invoke(Class targetClass, String methodName, Object[] args) {
+  public Object invoke(final Class targetClass, final String methodName, final Object[] args) {
     if (!this.available) {
-      String s = "VM not available: " + this;
-      throw new RMIException(this, targetClass.getName(), methodName,
-            new IllegalStateException(s));
-    }
-    MethExecutorResult result = null;
-    int retryCount = 120;
-    do {
-    try {
-      result = this.client.executeMethodOnClass(targetClass.getName(), methodName, args);
-      break; // out of while loop
-    } catch( RemoteException e ) {
-      boolean isWindows = false;
-      String os = System.getProperty("os.name");
-      if (os != null) {
-        if (os.indexOf("Windows") != -1) {
-          isWindows = true;
-        }
-      }
-      if (isWindows && retryCount-- > 0) {
-        boolean interrupted = Thread.interrupted();
-        try { Thread.sleep(1000); } catch (InterruptedException ignore) {interrupted = true;}
-        finally {
-          if (interrupted) {
-            Thread.currentThread().interrupt();
-          }
-        }
-      } else {
-        throw new RMIException(this, targetClass.getName(), methodName, e );
-      }
+      throw new RMIException(this, targetClass.getName(), methodName, new IllegalStateException("VM not available: " + this));
     }
-    } while (true);
+
+    MethExecutorResult result = execute(targetClass, methodName, args);
 
     if (!result.exceptionOccurred()) {
       return result.getResult();
 
     } else {
-      Throwable thr = result.getException();
-      throw new RMIException(this, targetClass.getName(), methodName, thr,
-                             result.getStackTrace()); 
+      throw new RMIException(this, targetClass.getName(), methodName, result.getException(), result.getStackTrace()); 
     }
   }
 
   /**
-   * Asynchronously invokes a static method with an {@link Object} or
-   * <code>void</code> return type in this VM.  If the return type of
-   * the method is <code>void</code>, <code>null</code> is returned.
+   * Asynchronously invokes an instance method with an {@link Object} or 
+   * {@code void} return type in this {@code VM}.  If the return type of the
+   * method is {@code void}, {@code null} is returned.
    *
-   * @param targetClass
-   *        The class on which to invoke the method
-   * @param methodName
-   *        The name of the method to invoke
-   * @param args
-   *        Arguments passed to the method call (must be {@link
-   *        java.io.Serializable}).
-   * @deprecated Use {@link #invoke(SerializableCallableIF)} instead 
+   * @param  targetObject
+   *         The object on which to invoke the method
+   * @param  methodName
+   *         The name of the method to invoke
+   * @param  args
+   *         Arguments passed to the method call (must be {@link
+   *         java.io.Serializable}).
+   *         
+   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead
    */
-  public AsyncInvocation invokeAsync(final Class targetClass, 
-                                     final String methodName,
-                                     final Object[] args) {
-    AsyncInvocation ai =
-      new AsyncInvocation(targetClass, methodName, new Runnable() {
-        public void run() {
-          final Object o = invoke(targetClass, methodName, args);
-          AsyncInvocation.setReturnValue(o);
-        }
-      });
-    ai.start();
-    return ai;
+  public AsyncInvocation invokeAsync(final Object targetObject, final String methodName, final Object[] args) {
+    return new AsyncInvocation(targetObject, methodName, () -> invoke(targetObject, methodName, args)).start();
   }
 
   /**
    * Asynchronously invokes an instance method with an {@link Object} or
-   * <code>void</code> return type in this VM.  If the return type of
-   * the method is <code>void</code>, <code>null</code> is returned.
+   * {@code void} return type in this {@code VM}.  If the return type of the
+   * method is {@code void}, {@code null} is returned.
+   *
+   * @param  targetClass
+   *         The class on which to invoke the method
+   * @param  methodName
+   *         The name of the method to invoke
+   * @param  args
+   *         Arguments passed to the method call (must be {@link
+   *         java.io.Serializable}).
    *
-   * @param o
-   *        The object on which to invoke the method
-   * @param methodName
-   *        The name of the method to invoke
-   * @param args
-   *        Arguments passed to the method call (must be {@link
-   *        java.io.Serializable}).
-   * @deprecated Use {@link #invoke(SerializableCallableIF)} instead
+   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead
    */
-  public AsyncInvocation invokeAsync(final Object o, 
-                                     final String methodName,
-                                     final Object[] args) {
-    AsyncInvocation ai =
-      new AsyncInvocation(o, methodName, new Runnable() {
-        public void run() {
-          final Object ret = invoke(o, methodName, args);
-          AsyncInvocation.setReturnValue(ret);
-        }
-      });
-    ai.start();
-    return ai;
+  public AsyncInvocation invokeAsync(final Class<?> targetClass, final String methodName, final Object[] args) {
+    return new AsyncInvocation(targetClass, methodName, () -> invoke(targetClass, methodName, args)).start();
   }
 
   /**
-   * Invokes the <code>run</code> method of a {@link Runnable} in this
-   * VM.  Recall that <code>run</code> takes no arguments and has no
-   * return value.
-   *
-   * @param r
-   *        The <code>Runnable</code> to be run
+   * Invokes the {@code run} method of a {@link Runnable} in this VM.  Recall
+   * that {@code run} takes no arguments and has no return value.
    *
+   * @param  runnable
+   *         The {@code Runnable} to be run
+   *         
    * @see SerializableRunnable
    */
-  public AsyncInvocation invokeAsync(SerializableRunnableIF r) {
-    return invokeAsync(r, "run", new Object[0]);
+  public AsyncInvocation invokeAsync(final SerializableRunnableIF runnable) {
+    return invokeAsync(runnable, "run", new Object[0]);
   }
   
   /**
-   * Invokes the <code>run</code> method of a {@link Runnable} in this
-   * VM.  Recall that <code>run</code> takes no arguments and has no
-   * return value.  The Runnable is wrapped in a NamedRunnable having
-   * the given name so it shows up in dunit logs.
+   * Invokes the {@code run} method of a {@link Runnable} in this VM.  Recall 
+   * that {@code run} takes no arguments and has no return value.  The 
+   * {@code Runnable} is wrapped in a {@link NamedRunnable} having the given 
+   * name so it shows up in DUnit logs.
    *
-   * @param r
-   *        The <code>Runnable</code> to be run
-   * @param name the name of the runnable, which will be logged in dunit output
+   * @param  runnable
+   *         The {@code Runnable} to be run
+   * @param  name 
+   *         The name of the {@code Runnable}, which will be logged in DUnit 
+   *         output
    *
    * @see SerializableRunnable
    */
-  public AsyncInvocation invokeAsync(String name, SerializableRunnableIF r) {
-    NamedRunnable nr = new NamedRunnable(name, r);
-    return invokeAsync(nr, "run", new Object[0]);
+  public AsyncInvocation invokeAsync(final String name, final SerializableRunnableIF runnable) {
+    return invokeAsync(new NamedRunnable(name, runnable), "run", new Object[0]);
   }
   
   /**
-   * Invokes the <code>call</code> method of a {@link Runnable} in this
-   * VM.  
+   * Invokes the {@code call} method of a {@link Callable} in this {@code VM}.  
    *
-   * @param c
-   *        The <code>Callable</code> to be run
-   * @param name the name of the callable, which will be logged in dunit output
+   * @param  callable
+   *         The {@code Callable} to be run
+   * @param  name 
+   *         The name of the {@code Callable}, which will be logged in dunit 
+   *         output
    *
    * @see SerializableCallable
    */
-  public <T> AsyncInvocation<T> invokeAsync(String name, SerializableCallableIF<T> c) {
-    return invokeAsync(new NamedCallable(name, c), "call", new Object[0]);
+  public <T> AsyncInvocation<T> invokeAsync(final String name, final SerializableCallableIF<T> callable) {
+    return invokeAsync(new NamedCallable(name, callable), "call", new Object[0]);
   }
 
   /**
-   * Invokes the <code>call</code> method of a {@link Runnable} in this
-   * VM.  
+   * Invokes the {@code call} method of a {@link Callable} in this {@code VM}.
    *
-   * @param c
-   *        The <code>Callable</code> to be run
+   * @param  callable
+   *         The {@code Callable} to be run
    *
    * @see SerializableCallable
    */
-  public <T> AsyncInvocation<T> invokeAsync(SerializableCallableIF<T> c) {
-    return invokeAsync(c, "call", new Object[0]);
+  public <T> AsyncInvocation<T> invokeAsync(final SerializableCallableIF<T> callable) {
+    return invokeAsync(callable, "call", new Object[0]);
   }
 
   /**
-   * Invokes the <code>run</code> method of a {@link Runnable} in this
-   * VM.  Recall that <code>run</code> takes no arguments and has no
-   * return value.
+   * Invokes the {@code run} method of a {@link Runnable} in this {@code VM}.
+   * Recall that {@code run} takes no arguments and has no return value.
    *
-   * @param r
-   *        The <code>Runnable</code> to be run
-   * @param name the name of the runnable, which will be logged in dunit output
+   * @param  runnable
+   *         The {@code Runnable} to be run
+   * @param  name 
+   *         The name of the {@code Runnable}, which will be logged in DUnit 
+   *         output
    *
    * @see SerializableRunnable
    */
-  public void invoke(String name, SerializableRunnableIF r) {
-    invoke(new NamedRunnable(name, r), "run");
+  public void invoke(final String name, final SerializableRunnableIF runnable) {
+    invoke(new NamedRunnable(name, runnable), "run");
   }
 
   /**
-   * Invokes the <code>run</code> method of a {@link Runnable} in this
-   * VM.  Recall that <code>run</code> takes no arguments and has no
-   * return value.
+   * Invokes the {@code run} method of a {@link Runnable} in this {@code VM}.
+   * Recall that {@code run} takes no arguments and has no return value.
    *
-   * @param r
-   *        The <code>Runnable</code> to be run
+   * @param  runnable
+   *         The {@code Runnable} to be run
    *
    * @see SerializableRunnable
    */
-  public void invoke(SerializableRunnableIF r) {
-    invoke(r, "run");
+  public void invoke(final SerializableRunnableIF runnable) {
+    invoke(runnable, "run");
   }
   
   /**
-   * Invokes the <code>run</code> method of a {@link Runnable} in this
-   * VM.  Recall that <code>run</code> takes no arguments and has no
-   * return value.
+   * Invokes the {@code call} method of a {@link Callable} in this {@code VM}.
    *
-   * @param c
-   *        The <code>Callable</code> to be run
-   * @param name the name of the callable, which will be logged in dunit output
+   * @param  callable
+   *         The {@code Callable} to be run
+   * @param  name 
+   *         The name of the {@code Callable}, which will be logged in DUnit 
+   *         output
    *
    * @see SerializableCallable
    */
-  public <T>  T invoke(String name, SerializableCallableIF<T> c) {
-    return (T) invoke(new NamedCallable(name, c), "call");
+  public <T>  T invoke(final String name, final SerializableCallableIF<T> callable) {
+    return (T) invoke(new NamedCallable(name, callable), "call");
   }
   
   /**
-   * Invokes the <code>run</code> method of a {@link Runnable} in this
-   * VM.  Recall that <code>run</code> takes no arguments and has no
-   * return value.
+   * Invokes the {@code call} method of a {@link Callable} in this {@code VM}. 
    *
-   * @param c
-   *        The <code>Callable</code> to be run
+   * @param  callable
+   *         The {@code Callable} to be run
    *
    * @see SerializableCallable
    */
-  public <T>  T invoke(SerializableCallableIF<T> c) {
-    return (T) invoke(c, "call");
+  public <T>  T invoke(final SerializableCallableIF<T> callable) {
+    return (T) invoke(callable, "call");
   }
   
   /**
-   * Invokes the <code>run</code> method of a {@link Runnable} in this
-   * VM.  If the invocation throws AssertionFailedError, and repeatTimeoutMs
-   * is >0, the <code>run</code> method is invoked repeatedly until it
+   * Invokes the {@code run} method of a {@link Runnable} in this {@code VM}.
+   * If the invocation throws AssertionFailedError, and repeatTimeoutMs
+   * is >0, the {@code run} method is invoked repeatedly until it
    * either succeeds, or repeatTimeoutMs has passed.  The AssertionFailedError
-   * is thrown back to the sender of this method if <code>run</code> has not
+   * is thrown back to the sender of this method if {@code run} has not
    * completed successfully before repeatTimeoutMs has passed.
    * 
-   * @deprecated Please use {@link com.jayway.awaitility.Awaitility} with {@link #invoke(SerializableCallableIF)} instead.
+   * @deprecated Please use {@link Awaitility} to await condition and then {@link #invoke(SerializableCallableIF)} instead.
    */
-  public void invokeRepeatingIfNecessary(RepeatableRunnable o, long repeatTimeoutMs) {
-    invoke(o, "runRepeatingIfNecessary", new Object[] {new Long(repeatTimeoutMs)});
+  public void invokeRepeatingIfNecessary(final RepeatableRunnable runnable, final long repeatTimeoutMs) {
+    invoke(runnable, "runRepeatingIfNecessary", new Object[] { repeatTimeoutMs });
   }
 
   /**
-   * Invokes an instance method with no arguments on an object that is
-   * serialized into this VM.  The return type of the method can be
-   * either {@link Object} or <code>void</code>.  If the return type
-   * of the method is <code>void</code>, <code>null</code> is
-   * returned.
+   * Invokes an instance method with no arguments on an object that is 
+   * serialized into this {@code VM}.  The return type of the method can be
+   * either {@link Object} or {@code void}.  If the return type of the method
+   * is {@code void}, {@code null} is returned.
    *
-   * @param o
-   *        The receiver of the method invocation
-   * @param methodName
-   *        The name of the method to invoke
+   * @param  targetObject
+   *         The receiver of the method invocation
+   * @param  methodName
+   *         The name of the method to invoke
    *
    * @throws RMIException
-   *         An exception occurred on while invoking the method in
-   *         this VM
-   * @deprecated Use {@link #invoke(SerializableCallableIF)} instead        
+   *         Wraps any underlying exception thrown while invoking the method in
+   *         this {@code VM}
+   *         
+   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead.        
    */
-  public Object invoke(Object o, String methodName) {
-    return invoke(o, methodName, new Object[0]);
+  public Object invoke(final Object targetObject, final String methodName) {
+    return invoke(targetObject, methodName, new Object[0]);
   }
   
   /**
-   * Invokes an instance method on an object that is serialized into
-   * this VM.  The return type of the method can be either {@link
-   * Object} or <code>void</code>.  If the return type of the method
-   * is <code>void</code>, <code>null</code> is returned.
+   * Invokes an instance method on an object that is serialized into this
+   * {@code VM}.  The return type of the method can be either {@link Object} or
+   * {@code void}.  If the return type of the method is {@code void},
+   * {@code null} is returned.
    *
-   * @param o
-   *        The receiver of the method invocation
-   * @param methodName
-   *        The name of the method to invoke
-   * @param args
-   *        Arguments passed to the method call (must be {@link
-   *        java.io.Serializable}). 
+   * @param  targetObject
+   *         The receiver of the method invocation
+   * @param  methodName
+   *         The name of the method to invoke
+   * @param  args
+   *         Arguments passed to the method call (must be {@link
+   *         java.io.Serializable}).
    *
    * @throws RMIException
-   *         An exception occurred on while invoking the method in
-   *         this VM
-   * @deprecated Use {@link #invoke(SerializableCallableIF)} instead        
+   *         Wraps any underlying exception thrown while invoking the method in
+   *         this {@code VM}
+   *
+   * @deprecated Please use {@link #invoke(SerializableCallableIF)} instead.
    */
-  public Object invoke(Object o, String methodName, Object[] args) {
+  public Object invoke(final Object targetObject, final String methodName, final Object[] args) {
     if (!this.available) {
-      String s = "VM not available: " + this;
-      throw new RMIException(this, o.getClass().getName(), methodName,
-            new IllegalStateException(s));
+      throw new RMIException(this, targetObject.getClass().getName(), methodName, new IllegalStateException("VM not available: " + this));
     }
-    MethExecutorResult result = null;
-    int retryCount = 120;
-    do {
-    try {
-      if ( args == null )
-        result = this.client.executeMethodOnObject(o, methodName);
-      else
-        result = this.client.executeMethodOnObject(o, methodName, args);
-      break; // out of while loop
-    } catch( RemoteException e ) {
-      if (retryCount-- > 0) {
-        boolean interrupted = Thread.interrupted();
-        try { Thread.sleep(1000); } catch (InterruptedException ignore) {interrupted = true;}
-        finally {
-          if (interrupted) {
-            Thread.currentThread().interrupt();
-          }
-        }
-      } else {
-        throw new RMIException(this, o.getClass().getName(), methodName, e );
-      }
-    }
-    } while (true);
+
+    MethExecutorResult result = execute(targetObject, methodName, args);
 
     if (!result.exceptionOccurred()) {
       return result.getResult();
 
     } else {
-      Throwable thr = result.getException();
-      throw new RMIException(this, o.getClass().getName(), methodName, thr,
-                             result.getStackTrace()); 
+      throw new RMIException(this, targetObject.getClass().getName(), methodName, result.getException(), result.getStackTrace());
     }
   }
 
-
-
-
   /**
-   * Synchronously bounces (mean kills and restarts) this <code>VM</code>.
-   * Concurrent bounce attempts are synchronized but attempts to invoke
-   * methods on a bouncing VM will cause test failure.  Tests using bounce
-   * should be placed at the end of the dunit test suite, since an exception
-   * here will cause all tests using the unsuccessfully bounced VM to fail.
+   * Synchronously bounces (mean kills and restarts) this {@code VM}.
+   * Concurrent bounce attempts are synchronized but attempts to invoke methods
+   * on a bouncing {@code VM} will cause test failure.  Tests using bounce
+   * should be placed at the end of the DUnit test suite, since an exception
+   * here will cause all tests using the unsuccessfully bounced {@code VM} to
+   * fail.
    * 
-   * This method is currently not supported by the standalone dunit
-   * runner.
+   * This method is currently not supported by the standalone DUnit runner.
    *
-   * @throws RMIException if an exception occurs while bouncing this VM, for
-   *  example a HydraTimeoutException if the VM fails to stop within 
-   *  hydra.Prms#maxClientShutdownWaitSec or restart within 
-   *  hydra.Prms#maxClientStartupWaitSec.
+   * @throws RMIException if an exception occurs while bouncing this
+   *         {@code VM}, for example a {@code HydraTimeoutException} if the
+   *         {@code VM} fails to stop within
+   *         {@code hydra.Prms#maxClientShutdownWaitSec} or restart within
+   *         {@code hydra.Prms#maxClientStartupWaitSec}.
    */
   public synchronized void bounce() {
     if (!this.available) {
-      String s = "VM not available: " + this;
-      throw new RMIException(this, this.getClass().getName(), "bounceVM",
-            new IllegalStateException(s));
+      throw new RMIException(this, getClass().getName(), "bounceVM", new IllegalStateException("VM not available: " + this));
     }
+
     this.available = false;
+
     try {
       BounceResult result = DUnitEnv.get().bounce(this.pid);
-      
       this.pid = result.getNewPid();
       this.client = result.getNewClient();
       this.available = true;
+
     } catch (UnsupportedOperationException e) {
       this.available = true;
       throw e;
+
     } catch (RemoteException e) {
       StringWriter sw = new StringWriter();
       e.printStackTrace(new PrintWriter(sw, true));
-      RMIException rmie = new RMIException(this, this.getClass().getName(),
-        "bounceVM", e, sw.toString());
+      RMIException rmie = new RMIException(this, getClass().getName(), "bounceVM", e, sw.toString());
       throw rmie;
     }
   }
 
-  /////////////////////  Utility Methods  ////////////////////
-
   public String toString() {
-    return "VM " + this.getPid() + " running on " + this.getHost();
+    return "VM " + getPid() + " running on " + getHost();
   }
 
-  public static int getCurrentVMNum() {
-    return DUnitEnv.get().getVMID();
-  }
-  
   public File getWorkingDirectory() {
-    return DUnitEnv.get().getWorkingDirectory(this.getPid());
+    return DUnitEnv.get().getWorkingDirectory(getPid());
   }
 
-  /** Return the total number of VMs on all hosts */
-  public static int getVMCount() {
-    int count = 0;
-    for (int h = 0; h < Host.getHostCount(); h++) {
-      Host host = Host.getHost(h);
-      count += host.getVMCount();
+  private MethExecutorResult execute(final Class targetClass, final String methodName, final Object[] args) {
+    try {
+      return this.client.executeMethodOnClass(targetClass.getName(), methodName, args);
+    } catch (RemoteException exception) {
+      throw new RMIException(this, targetClass.getName(), methodName, exception);
     }
-    return count;
   }
 
+  private MethExecutorResult execute(final Object targetObject, final String methodName, final Object[] args) {
+    try {
+      if (args == null) {
+        return this.client.executeMethodOnObject(targetObject, methodName);
+      } else {
+        return this.client.executeMethodOnObject(targetObject, methodName, args);
+      }
+    } catch (RemoteException exception) {
+      throw new RMIException(this, targetObject.getClass().getName(), methodName, exception);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Wait.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Wait.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Wait.java
index b73a25c..564fb91 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Wait.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/Wait.java
@@ -16,8 +16,8 @@
  */
 package com.gemstone.gemfire.test.dunit;
 
-import static org.junit.Assert.fail;
 import static com.gemstone.gemfire.test.dunit.Jitter.*;
+import static org.junit.Assert.*;
 
 import org.apache.logging.log4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java
index b5bd5f7..6a719ab 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/cache/internal/JUnit4CacheTestCase.java
@@ -18,13 +18,14 @@ package com.gemstone.gemfire.test.dunit.cache.internal;
 
 import java.io.File;
 import java.io.FileWriter;
-import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.Arrays;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.logging.log4j.Logger;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
@@ -56,7 +57,6 @@ 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.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.logging.log4j.Logger;
 
 /**
  * This class is the base class for all distributed tests using JUnit 4 that

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit3DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit3DistributedTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit3DistributedTestCase.java
index f453e7e..71dee9b 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit3DistributedTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/internal/JUnit3DistributedTestCase.java
@@ -19,14 +19,15 @@ package com.gemstone.gemfire.test.dunit.internal;
 import java.io.Serializable;
 import java.util.Properties;
 
+import junit.framework.TestCase;
+import org.apache.logging.log4j.Logger;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import junit.framework.TestCase;
-import org.apache.logging.log4j.Logger;
-import org.junit.experimental.categories.Category;
 
 /**
  * This class is the superclass of all distributed tests using JUnit 3.
@@ -36,7 +37,7 @@ public abstract class JUnit3DistributedTestCase extends TestCase implements Dist
 
   private static final Logger logger = LogService.getLogger();
 
-  private final JUnit4DistributedTestCase delegate = new JUnit4DistributedTestCase(this);
+  private final JUnit4DistributedTestCase delegate = new JUnit4DistributedTestCase(this) {};
 
   /**
    * Constructs a new distributed test. All JUnit 3 test classes need to have a



[13/32] incubator-geode git commit: GEODE-1225: Converted SSL log message from info to fine

Posted by ji...@apache.org.
GEODE-1225: Converted SSL log message from info to fine


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

Branch: refs/heads/feature/GEODE-17-2
Commit: bbf705edc7cef804d3b84c1b40551630ae67a5fb
Parents: 7d5f39a
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Wed Apr 13 14:10:18 2016 -0700
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Wed Apr 13 14:10:18 2016 -0700

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/internal/SocketCreator.java    | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bbf705ed/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
index 2130a26..acdfbc7 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/SocketCreator.java
@@ -1055,7 +1055,9 @@ public class SocketCreator {
             sslSocket.startHandshake();
             SSLSession session = sslSocket.getSession();
             Certificate[] peer = session.getPeerCertificates();
-            logger.info(LocalizedMessage.create(LocalizedStrings.SocketCreator_SSL_CONNECTION_FROM_PEER_0, ((X509Certificate)peer[0]).getSubjectDN()));
+            if (logger.isDebugEnabled()) {
+              logger.debug(LocalizedMessage.create(LocalizedStrings.SocketCreator_SSL_CONNECTION_FROM_PEER_0, ((X509Certificate)peer[0]).getSubjectDN()));
+            }
           }
           catch (SSLPeerUnverifiedException ex) {
             if (this.needClientAuth) {
@@ -1119,7 +1121,9 @@ public class SocketCreator {
         sslSocket.startHandshake();
         SSLSession session = sslSocket.getSession();
         Certificate[] peer = session.getPeerCertificates();
-        logger.info(LocalizedMessage.create(LocalizedStrings.SocketCreator_SSL_CONNECTION_FROM_PEER_0, ((X509Certificate)peer[0]).getSubjectDN()));
+        if (logger.isDebugEnabled()) {
+          logger.debug(LocalizedMessage.create(LocalizedStrings.SocketCreator_SSL_CONNECTION_FROM_PEER_0, ((X509Certificate)peer[0]).getSubjectDN()));
+        }
       }
       catch (SSLPeerUnverifiedException ex) {
         if (this.needClientAuth) {


[27/32] incubator-geode git commit: GEODE-1225: Set base log level DEBUG in JSSESocketJUnitTest

Posted by ji...@apache.org.
GEODE-1225: Set base log level DEBUG in  JSSESocketJUnitTest


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

Branch: refs/heads/feature/GEODE-17-2
Commit: ce889fecf8409edc5fb1306500737f94509176b0
Parents: d50f16f
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Thu Apr 14 13:57:09 2016 -0700
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Thu Apr 14 14:56:55 2016 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/internal/JSSESocketJUnitTest.java   | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ce889fec/geode-core/src/test/java/com/gemstone/gemfire/internal/JSSESocketJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/JSSESocketJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/JSSESocketJUnitTest.java
index c8927b6..63febc0 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/JSSESocketJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/JSSESocketJUnitTest.java
@@ -97,6 +97,11 @@ public class JSSESocketJUnitTest {
     
     TestAppender.create();
     
+    // Get original base log level
+    Level originalBaseLevel = LogService.getBaseLogLevel();
+    try {
+    // Set base log level to debug to log the SSL messages
+    LogService.setBaseLogLevel(Level.DEBUG);
     {
       System.setProperty( "gemfire.mcast-port", "0");
       System.setProperty( "gemfire.ssl-enabled", "true" );
@@ -146,6 +151,10 @@ public class JSSESocketJUnitTest {
     if ( peerLogCount != 2 ) {
       throw new Exception( "Expected to find to peer identities logged." );
     }
+    } finally {
+      // Reset original base log level
+      LogService.setBaseLogLevel(originalBaseLevel);
+    }
   }
   
   /** not actually related to this test class, but this is as good a place
@@ -230,7 +239,7 @@ public class JSSESocketJUnitTest {
       Appender appender = new TestAppender();
       Logger socketCreatorLogger = (Logger) LogManager.getLogger(SOCKET_CREATOR_CLASSNAME);
       LoggerConfig config = socketCreatorLogger.getContext().getConfiguration().getLoggerConfig(SOCKET_CREATOR_CLASSNAME);
-      config.addAppender(appender, Level.INFO, null);
+      config.addAppender(appender, Level.DEBUG, null);
       return appender;
     }
 


[09/32] incubator-geode git commit: GEODE-1216 Fix the scalability of remove member

Posted by ji...@apache.org.
GEODE-1216 Fix the scalability of remove member

Enable the optimization in sendSuspectRequest so that health monitor
sends suspect requests to appropriate recipients.


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 0f692f860c6db1f3368d474bfb0c6e8b916f833f
Parents: 9d20f22
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Wed Apr 13 10:22:12 2016 -0700
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Wed Apr 13 10:22:12 2016 -0700

----------------------------------------------------------------------
 .../membership/gms/fd/GMSHealthMonitor.java      | 19 ++++++++-----------
 1 file changed, 8 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0f692f86/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
index 5427d77..2d0f039 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
@@ -1331,18 +1331,15 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
 //    }
     logger.debug("Sending suspect request for members {}", requests);
     List<InternalDistributedMember> recipients;
-//  TODO this needs some rethinking - we need the guys near the
-//  front of the membership view who aren't preferred for coordinator
-//  to see the suspect message.
-//    if (v.size() > 20) {
-//      HashSet<InternalDistributedMember> filter = new HashSet<InternalDistributedMember>();
-//      for (int i = 0; i < requests.size(); i++) {
-//        filter.add(requests.get(i).getSuspectMember());
-//      }
-//      recipients = currentView.getPreferredCoordinators(filter, services.getJoinLeave().getMemberID(), 5);
-//    } else {
+    if (currentView.size() > 4) {
+      HashSet<InternalDistributedMember> filter = new HashSet<InternalDistributedMember>();
+      for (int i = 0; i < requests.size(); i++) {
+        filter.add(requests.get(i).getSuspectMember());
+      }
+      recipients = currentView.getPreferredCoordinators(filter, services.getJoinLeave().getMemberID(), 5);
+    } else {
       recipients = currentView.getMembers();
-//    }
+    }
 
     SuspectMembersMessage smm = new SuspectMembersMessage(recipients, requests);
     Set<InternalDistributedMember> failedRecipients;


[26/32] incubator-geode git commit: GEODE-1230: add missing junit category and remove it where it's not needed

Posted by ji...@apache.org.
GEODE-1230: add missing junit category and remove it where it's not needed


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

Branch: refs/heads/feature/GEODE-17-2
Commit: d50f16ffd760bbf26cf2f518be285887b3b5da27
Parents: f2d5969
Author: Kirk Lund <kl...@pivotal.io>
Authored: Thu Apr 14 14:17:11 2016 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Thu Apr 14 14:17:11 2016 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java     | 3 +++
 .../test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java        | 1 -
 .../gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.java      | 1 -
 .../JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java     | 3 +++
 .../tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java     | 3 +++
 5 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d50f16ff/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
index 17afcc6..a5663e5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/BasicDUnitTest.java
@@ -24,12 +24,15 @@ import static org.hamcrest.Matchers.*;
 
 import java.util.Properties;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.DUnitEnv;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.RMIException;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * This class tests the basic functionality of the distributed unit

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d50f16ff/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java
index c92ab3d..c4f6160 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetDefaultDiskStoreNameDUnitTest.java
@@ -26,7 +26,6 @@ import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 @SuppressWarnings("serial")
-@Category(DistributedTest.class)
 public class GetDefaultDiskStoreNameDUnitTest extends DistributedTestCase {
 
   public GetDefaultDiskStoreNameDUnitTest(final String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d50f16ff/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.java
index 13550f6..2e9b1fd 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/GetTestMethodNameDUnitTest.java
@@ -25,7 +25,6 @@ import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 @SuppressWarnings("serial")
-@Category(DistributedTest.class)
 public class GetTestMethodNameDUnitTest extends DistributedTestCase {
 
   public GetTestMethodNameDUnitTest(final String name) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d50f16ff/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
index b03c42a..113099d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest.java
@@ -22,14 +22,17 @@ import static com.gemstone.gemfire.test.dunit.Invoke.*;
 import java.util.Properties;
 
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Verifies that overriding {@code getDistributedSystemProperties} results
  * in {@code disconnectAllFromDS} during tear down.
  */
+@Category(DistributedTest.class)
 public class JUnit4OverridingGetPropertiesDisconnectsAllDUnitTest extends JUnit4DistributedTestCase {
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d50f16ff/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java
index 776fb3d..fb4b4fe 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/tests/OverridingGetPropertiesDisconnectsAllDUnitTest.java
@@ -20,8 +20,11 @@ import static com.gemstone.gemfire.test.dunit.Invoke.*;
 
 import java.util.Properties;
 
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Verifies that overriding {@code getDistributedSystemProperties} results


[18/32] incubator-geode git commit: GEODE-1162: convert all CLI command DUnit tests to JUnit 4

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStartDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStartDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStartDUnitTest.java
index 6f26e3b..5edd3d2 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStartDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStartDUnitTest.java
@@ -16,11 +16,16 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
 
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.management.cli.Result;
@@ -29,20 +34,18 @@ import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
-public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
+@Category(DistributedTest.class)
+public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public WanCommandGatewayReceiverStartDUnitTest(String name) {
-    super(name);
-  }
-
   /**
    * Test wan commands for error in input 1> start gateway-sender command needs
    * only one of member or group.
    */
+  @Test
   public void testStartGatewayReceiver_ErrorConditions() {
 
     VM puneLocator = Host.getLocator();
@@ -66,7 +69,7 @@ public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewayReceiver_ErrorConditions stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.ERROR, cmdResult.getStatus());
       assertTrue(strCmdResult.contains(CliStrings.PROVIDE_EITHER_MEMBER_OR_GROUP_MESSAGE));
@@ -75,6 +78,7 @@ public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
     }
   }
 
+  @Test
   public void testStartGatewayReceiver() {
     
     VM puneLocator = Host.getLocator();
@@ -95,12 +99,12 @@ public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
     vm4.invoke(() -> WANCommandTestBase.verifyReceiverState( false ));
     vm5.invoke(() -> WANCommandTestBase.verifyReceiverState( false ));
     
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.START_GATEWAYRECEIVER;
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       
       TabularResultData resultData = (TabularResultData) cmdResult.getResultData();
@@ -120,6 +124,7 @@ public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
    * test to validate that the start gateway sender starts the gateway sender on
    * a member
    */
+  @Test
   public void testStartGatewayReceiver_onMember() {
 
     VM puneLocator = Host.getLocator();
@@ -141,14 +146,14 @@ public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
     vm5.invoke(() -> WANCommandTestBase.verifyReceiverState( false ));
     
     final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.START_GATEWAYRECEIVER + " --"
         + CliStrings.START_GATEWAYRECEIVER__MEMBER+ "=" + vm1Member.getId();
     
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewayReceiver_onMember stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       assertTrue(strCmdResult.contains("is started on member"));
@@ -165,6 +170,7 @@ public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
    * test to validate that the start gateway sender starts the gateway sender on
    * a group of members
    */
+  @Test
   public void testStartGatewayReceiver_Group() {
 
     VM puneLocator = Host.getLocator();
@@ -185,13 +191,13 @@ public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
     vm4.invoke(() -> WANCommandTestBase.verifyReceiverState( false ));
     vm5.invoke(() -> WANCommandTestBase.verifyReceiverState( false ));
     
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.START_GATEWAYRECEIVER + " --"
         + CliStrings.START_GATEWAYRECEIVER__GROUP + "=RG1";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewayReceiver_Group stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -215,6 +221,7 @@ public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
    * sender members belongs to multiple groups
    * 
    */
+  @Test
   public void testStartGatewayReceiver_MultipleGroup() {
     
     VM puneLocator = Host.getLocator();
@@ -239,13 +246,13 @@ public class WanCommandGatewayReceiverStartDUnitTest extends WANCommandTestBase{
     vm6.invoke(() -> WANCommandTestBase.verifyReceiverState( false ));
     vm7.invoke(() -> WANCommandTestBase.verifyReceiverState( false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.START_GATEWAYRECEIVER + " --"
         + CliStrings.START_GATEWAYRECEIVER__GROUP + "=RG1,RG2";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewayReceiver_Group stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStopDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStopDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStopDUnitTest.java
index 82ca3ba..d7bb6c3 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStopDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewayReceiverStopDUnitTest.java
@@ -16,11 +16,16 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
 
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.management.cli.Result;
@@ -29,20 +34,18 @@ import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+@Category(DistributedTest.class)
 public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public WanCommandGatewayReceiverStopDUnitTest(String name) {
-    super(name);
-  }
-
   /**
    * Test wan commands for error in input 1> start gateway-sender command needs
    * only one of member or group.
    */
+  @Test
   public void testStopGatewayReceiver_ErrorConditions() {
 
     VM puneLocator = Host.getLocator();
@@ -66,7 +69,7 @@ public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStopGatewayReceiver_ErrorConditions stringResult : "
               + strCmdResult + ">>>>");
       assertEquals(Result.Status.ERROR, cmdResult.getStatus());
@@ -76,6 +79,7 @@ public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
     }
   }
 
+  @Test
   public void testStopGatewayReceiver() {
 
     VM puneLocator = Host.getLocator();
@@ -96,12 +100,12 @@ public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
     vm4.invoke(() -> WANCommandTestBase.verifyReceiverState( true ));
     vm5.invoke(() -> WANCommandTestBase.verifyReceiverState( true ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STOP_GATEWAYRECEIVER;
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStopGatewayReceiver stringResult : " + strCmdResult + ">>>>");
 
       TabularResultData resultData = (TabularResultData) cmdResult
@@ -123,6 +127,7 @@ public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
    * test to validate that the start gateway sender starts the gateway sender on
    * a member
    */
+  @Test
   public void testStopGatewayReceiver_onMember() {
 
     VM puneLocator = Host.getLocator();
@@ -144,14 +149,14 @@ public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.verifyReceiverState( true ));
 
     final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STOP_GATEWAYRECEIVER + " --"
         + CliStrings.STOP_GATEWAYRECEIVER__MEMBER + "=" + vm1Member.getId();
 
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStopGatewayReceiver_onMember stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -169,6 +174,7 @@ public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
    * test to validate that the start gateway sender starts the gateway sender on
    * a group of members
    */
+  @Test
   public void testStopGatewayReceiver_Group() {
 
     VM puneLocator = Host.getLocator();
@@ -189,13 +195,13 @@ public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
     vm4.invoke(() -> WANCommandTestBase.verifyReceiverState( true ));
     vm5.invoke(() -> WANCommandTestBase.verifyReceiverState( true ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STOP_GATEWAYRECEIVER + " --"
         + CliStrings.STOP_GATEWAYRECEIVER__GROUP + "=RG1";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStopGatewayReceiver_Group stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -220,6 +226,7 @@ public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
    * sender members belongs to multiple groups
    * 
    */
+  @Test
   public void testStopGatewayReceiver_MultipleGroup() {
 
     VM puneLocator = Host.getLocator();
@@ -244,13 +251,13 @@ public class WanCommandGatewayReceiverStopDUnitTest extends WANCommandTestBase {
     vm6.invoke(() -> WANCommandTestBase.verifyReceiverState( true ));
     vm7.invoke(() -> WANCommandTestBase.verifyReceiverState( true ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STOP_GATEWAYRECEIVER + " --"
         + CliStrings.STOP_GATEWAYRECEIVER__GROUP + "=RG1,RG2";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStopGatewayReceiver_Group stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStartDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStartDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStartDUnitTest.java
index a10a926..ecc6cec 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStartDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStartDUnitTest.java
@@ -16,11 +16,16 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
 
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.management.cli.Result;
@@ -28,20 +33,18 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+@Category(DistributedTest.class)
 public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public WanCommandGatewaySenderStartDUnitTest(String name) {
-    super(name);
-  }
-
   /**
    * Test wan commands for error in input 1> start gateway-sender command needs
    * only one of member or group.
    */
+  @Test
   public void testStartGatewaySender_ErrorConditions() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -67,7 +70,7 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.ERROR, cmdResult.getStatus());
       assertTrue(strCmdResult.contains(CliStrings.PROVIDE_EITHER_MEMBER_OR_GROUP_MESSAGE));
@@ -86,6 +89,7 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
     }
   }
 
+  @Test
   public void testStartGatewaySender() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -115,13 +119,13 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", false, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.START_GATEWAYSENDER + " --"
         + CliStrings.START_GATEWAYSENDER__ID + "=ln";
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       
@@ -146,6 +150,7 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
    * test to validate that the start gateway sender starts the gateway sender on
    * a member
    */
+  @Test
   public void testStartGatewaySender_onMember() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -166,14 +171,14 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
         "ln", false, false ));
 
     final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.START_GATEWAYSENDER + " --"
         + CliStrings.START_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.START_GATEWAYSENDER__MEMBER + "=" + vm1Member.getId();
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       assertTrue(strCmdResult.contains("is started on member"));
@@ -189,6 +194,7 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
    * test to validate that the start gateway sender starts the gateway sender on
    * a group of members
    */
+  @Test
   public void testStartGatewaySender_Group() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -221,14 +227,14 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", false, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.START_GATEWAYSENDER + " --"
         + CliStrings.START_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.START_GATEWAYSENDER__GROUP + "=SenderGroup1";
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewaySender_Group stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -253,8 +259,8 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
   /**
    * Test to validate the scenario gateway sender is started when one or more
    * sender members belongs to multiple groups
-   * 
    */
+  @Test
   public void testStartGatewaySender_MultipleGroup() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -299,14 +305,14 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
     vm7.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", false, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.START_GATEWAYSENDER + " --"
         + CliStrings.START_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.START_GATEWAYSENDER__GROUP + "=SenderGroup1,SenderGroup2";
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewaySender_Group stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -334,8 +340,8 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
   /**
    * Test to validate the test scenario when one of the member ion group does
    * not have the sender.
-   * 
    */
+  @Test
   public void testStartGatewaySender_Group_MissingSenderFromGroup() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -364,7 +370,7 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", false, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.START_GATEWAYSENDER + " --"
         + CliStrings.START_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.START_GATEWAYSENDER__GROUP + "=SenderGroup1";
@@ -373,7 +379,7 @@ public class WanCommandGatewaySenderStartDUnitTest extends WANCommandTestBase {
       String strCmdResult = commandResultToString(cmdResult);
       assertTrue(strCmdResult.contains("Error"));
       assertTrue(strCmdResult.contains("is not available"));
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStartGatewaySender_Group stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStopDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStopDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStopDUnitTest.java
index f0e51bf..a90b984 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStopDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandGatewaySenderStopDUnitTest.java
@@ -16,11 +16,16 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
 
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.management.cli.Result;
@@ -28,16 +33,13 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+@Category(DistributedTest.class)
 public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public WanCommandGatewaySenderStopDUnitTest(String name) {
-    super(name);
-  }
-
   private CommandResult executeCommandWithIgnoredExceptions(String command) {
     final IgnoredException exln = IgnoredException.addIgnoredException("Could not connect");
     CommandResult commandResult =  executeCommand(command);
@@ -45,7 +47,7 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
     return commandResult;
   }
 
-
+  @Test
   public void testStopGatewaySender_ErrorConditions() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -71,7 +73,7 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStopGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.ERROR, cmdResult.getStatus());
       assertTrue(strCmdResult.contains(CliStrings.PROVIDE_EITHER_MEMBER_OR_GROUP_MESSAGE));
@@ -80,6 +82,7 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
     }
   }
 
+  @Test
   public void testStopGatewaySender() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -113,13 +116,13 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STOP_GATEWAYSENDER + " --"
         + CliStrings.STOP_GATEWAYSENDER__ID + "=ln";
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStopGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -145,6 +148,7 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
    * test to validate that the start gateway sender starts the gateway sender on
    * a member
    */
+  @Test
   public void testStopGatewaySender_onMember() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -167,14 +171,14 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
         "ln", true, false ));
 
     final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STOP_GATEWAYSENDER + " --"
         + CliStrings.STOP_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.STOP_GATEWAYSENDER__MEMBER + "=" + vm1Member.getId();
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testStopGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       assertTrue(strCmdResult.contains("is stopped on member"));
@@ -190,6 +194,7 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
    * test to validate that the start gateway sender starts the gateway sender on
    * a group of members
    */
+  @Test
   public void testStopGatewaySender_Group() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -226,14 +231,14 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STOP_GATEWAYSENDER + " --"
         + CliStrings.STOP_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.STOP_GATEWAYSENDER__GROUP + "=SenderGroup1";
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter()
+      getLogWriter()
           .info(
               "testStopGatewaySender_Group stringResult : " + strCmdResult
                   + ">>>>");
@@ -260,8 +265,8 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
   /**
    * Test to validate the scenario gateway sender is started when one or more
    * sender members belongs to multiple groups
-   * 
    */
+  @Test
   public void testStopGatewaySender_MultipleGroup() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -312,14 +317,14 @@ public class WanCommandGatewaySenderStopDUnitTest extends WANCommandTestBase {
     vm7.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STOP_GATEWAYSENDER + " --"
         + CliStrings.STOP_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.STOP_GATEWAYSENDER__GROUP + "=SenderGroup1,SenderGroup2";
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter()
+      getLogWriter()
           .info(
               "testStopGatewaySender_Group stringResult : " + strCmdResult
                   + ">>>>");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandListDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandListDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandListDUnitTest.java
index 8880348..bcab986 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandListDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandListDUnitTest.java
@@ -16,11 +16,16 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
 
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
@@ -28,16 +33,14 @@ import com.gemstone.gemfire.management.internal.cli.json.GfJsonException;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
-import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+@Category(DistributedTest.class)
 public class WanCommandListDUnitTest extends WANCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public WanCommandListDUnitTest(String name) {
-    super(name);
-  }
-
+  @Test
   public void testListGatewayWithNoSenderReceiver() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -54,19 +57,19 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
     vm4.invoke(() -> WANCommandTestBase.createCache( punePort ));
     vm5.invoke(() -> WANCommandTestBase.createCache( punePort ));
     
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.LIST_GATEWAY;
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info("testListGatewaySender : : " + strCmdResult);
+      getLogWriter().info("testListGatewaySender : : " + strCmdResult);
       assertEquals(Result.Status.ERROR, cmdResult.getStatus());
     } else {
       fail("testListGatewaySender failed as did not get CommandResult");
     }
   }
 
-  
+  @Test
   public void testListGatewaySender() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -98,12 +101,12 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.createSender(
         "ln_Serial", 2, false, 100, 400, false, false, null, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.LIST_GATEWAY;
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info("testListGatewaySender" + strCmdResult);
+      getLogWriter().info("testListGatewaySender" + strCmdResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       
       TabularResultData tableResultData =
@@ -119,6 +122,7 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
     }
   }
 
+  @Test
   public void testListGatewayReceiver() {
 
     Integer lnPort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -145,12 +149,12 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
     vm6.invoke(() -> WANCommandTestBase.createSender(
         "ln_Parallel", 1, true, 100, 400, false, false, null, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.LIST_GATEWAY;
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info("testListGatewayReceiver" + strCmdResult);
+      getLogWriter().info("testListGatewayReceiver" + strCmdResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       
       TabularResultData tableResultData =
@@ -167,7 +171,8 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
       fail("testListGatewayReceiver failed as did not get CommandResult");
     }
   }
-  
+
+  @Test
   public void testListGatewaySenderGatewayReceiver() throws GfJsonException {
 
     Integer lnPort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -202,13 +207,13 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
     vm7.invoke(() -> WANCommandTestBase.createSender(
         "ln_Parallel", 1, true, 100, 400, false, false, null, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.LIST_GATEWAY;
     CommandResult cmdResult = executeCommand(command);
     
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testListGatewaySenderGatewayReceiver : " + strCmdResult );
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       
@@ -228,7 +233,8 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
       fail("testListGatewaySenderGatewayReceiver failed as did not get CommandResult");
     }
   }
-  
+
+  @Test
   public void testListGatewaySenderGatewayReceiver_group() {
 
     Integer lnPort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -266,12 +272,12 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
     vm7.invoke(() -> WANCommandTestBase.createSender(
         "ln_Parallel", 1, true, 100, 400, false, false, null, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.LIST_GATEWAY + " --" + CliStrings.LIST_GATEWAY__GROUP + "=Serial_Sender";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testListGatewaySenderGatewayReceiver_group : " + strCmdResult );
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       
@@ -300,7 +306,7 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
       assertEquals(1, ports.size());
       
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testListGatewaySenderGatewayReceiver_group : " + strCmdResult );
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
@@ -311,7 +317,7 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
     cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testListGatewaySenderGatewayReceiver_group : " + strCmdResult );
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       
@@ -333,7 +339,7 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
     cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testListGatewaySenderGatewayReceiver_group : " + strCmdResult );
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       
@@ -354,7 +360,7 @@ public class WanCommandListDUnitTest extends WANCommandTestBase {
     cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testListGatewaySenderGatewayReceiver_group : " + strCmdResult );
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
index 360f2ab..b54b27f 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandPauseResumeDUnitTest.java
@@ -16,27 +16,30 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
 
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
-import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+@Category(DistributedTest.class)
 public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public WanCommandPauseResumeDUnitTest(String name) {
-    super(name);
-  }
-
+  @Test
   public void testPauseGatewaySender_ErrorConditions() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -62,7 +65,7 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testPauseGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.ERROR, cmdResult.getStatus());
       assertTrue(strCmdResult.contains(CliStrings.PROVIDE_EITHER_MEMBER_OR_GROUP_MESSAGE));
@@ -71,7 +74,12 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     }
   }
 
-  public void testPauseGatewaySender() {
+  /**
+   * test to validate that the start gateway sender starts the gateway sender on
+   * a member
+   */
+  @Test
+  public void testPauseGatewaySender_onMember() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
 
@@ -86,57 +94,34 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     vm3.invoke(() -> WANCommandTestBase.createCache( punePort ));
     vm3.invoke(() -> WANCommandTestBase.createSender( "ln",
         2, false, 100, 400, false, false, null, true ));
-    vm4.invoke(() -> WANCommandTestBase.createCache( punePort ));
-    vm4.invoke(() -> WANCommandTestBase.createSender( "ln",
-        2, false, 100, 400, false, false, null, true ));
-    vm5.invoke(() -> WANCommandTestBase.createCache( punePort ));
-    vm5.invoke(() -> WANCommandTestBase.createSender( "ln",
-        2, false, 100, 400, false, false, null, true ));
 
     vm3.invoke(() -> WANCommandTestBase.startSender( "ln" ));
-    vm4.invoke(() -> WANCommandTestBase.startSender( "ln" ));
-    vm5.invoke(() -> WANCommandTestBase.startSender( "ln" ));
 
     vm3.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, false ));
-    vm4.invoke(() -> WANCommandTestBase.verifySenderState(
-        "ln", true, false ));
-    vm5.invoke(() -> WANCommandTestBase.verifySenderState(
-        "ln", true, false ));
 
-    Wait.pause(10000);
+    final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
+    pause(10000);
     String command = CliStrings.PAUSE_GATEWAYSENDER + " --"
-        + CliStrings.PAUSE_GATEWAYSENDER__ID + "=ln";
+        + CliStrings.PAUSE_GATEWAYSENDER__ID + "=ln --"
+        + CliStrings.PAUSE_GATEWAYSENDER__MEMBER + "=" + vm1Member.getId();
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testPauseGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
-
-      TabularResultData resultData = (TabularResultData) cmdResult
-          .getResultData();
-      List<String> status = resultData.retrieveAllValues("Result");
-      assertEquals(5, status.size());
-      assertTrue(status.contains("Error"));
-      assertTrue(status.contains("OK"));
+      assertTrue(strCmdResult.contains("is paused on member"));
     } else {
       fail("testPauseGatewaySender failed as did not get CommandResult");
     }
 
     vm3.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, true ));
-    vm4.invoke(() -> WANCommandTestBase.verifySenderState(
-        "ln", true, true ));
-    vm5.invoke(() -> WANCommandTestBase.verifySenderState(
-        "ln", true, true ));
   }
 
-  /**
-   * test to validate that the start gateway sender starts the gateway sender on
-   * a member
-   */
-  public void testPauseGatewaySender_onMember() {
+  @Test
+  public void testPauseGatewaySender() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
 
@@ -150,37 +135,58 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
 
     vm3.invoke(() -> WANCommandTestBase.createCache( punePort ));
     vm3.invoke(() -> WANCommandTestBase.createSender( "ln",
-        2, false, 100, 400, false, false, null, true ));
+            2, false, 100, 400, false, false, null, true ));
+    vm4.invoke(() -> WANCommandTestBase.createCache( punePort ));
+    vm4.invoke(() -> WANCommandTestBase.createSender( "ln",
+            2, false, 100, 400, false, false, null, true ));
+    vm5.invoke(() -> WANCommandTestBase.createCache( punePort ));
+    vm5.invoke(() -> WANCommandTestBase.createSender( "ln",
+            2, false, 100, 400, false, false, null, true ));
 
     vm3.invoke(() -> WANCommandTestBase.startSender( "ln" ));
+    vm4.invoke(() -> WANCommandTestBase.startSender( "ln" ));
+    vm5.invoke(() -> WANCommandTestBase.startSender( "ln" ));
 
     vm3.invoke(() -> WANCommandTestBase.verifySenderState(
-        "ln", true, false ));
+            "ln", true, false ));
+    vm4.invoke(() -> WANCommandTestBase.verifySenderState(
+            "ln", true, false ));
+    vm5.invoke(() -> WANCommandTestBase.verifySenderState(
+            "ln", true, false ));
 
-    final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.PAUSE_GATEWAYSENDER + " --"
-        + CliStrings.PAUSE_GATEWAYSENDER__ID + "=ln --"
-        + CliStrings.PAUSE_GATEWAYSENDER__MEMBER + "=" + vm1Member.getId();
+            + CliStrings.PAUSE_GATEWAYSENDER__ID + "=ln";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
-          "testPauseGatewaySender stringResult : " + strCmdResult + ">>>>");
+      getLogWriter().info(
+              "testPauseGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
-      assertTrue(strCmdResult.contains("is paused on member"));
+
+      TabularResultData resultData = (TabularResultData) cmdResult
+              .getResultData();
+      List<String> status = resultData.retrieveAllValues("Result");
+      assertEquals(5, status.size());
+      assertTrue(status.contains("Error"));
+      assertTrue(status.contains("OK"));
     } else {
       fail("testPauseGatewaySender failed as did not get CommandResult");
     }
 
     vm3.invoke(() -> WANCommandTestBase.verifySenderState(
-        "ln", true, true ));
+            "ln", true, true ));
+    vm4.invoke(() -> WANCommandTestBase.verifySenderState(
+            "ln", true, true ));
+    vm5.invoke(() -> WANCommandTestBase.verifySenderState(
+            "ln", true, true ));
   }
 
   /**
    * test to validate that the start gateway sender starts the gateway sender on
    * a group of members
    */
+  @Test
   public void testPauseGatewaySender_Group() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -217,14 +223,14 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.PAUSE_GATEWAYSENDER + " --"
         + CliStrings.PAUSE_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.PAUSE_GATEWAYSENDER__GROUP + "=SenderGroup1";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testPauseGatewaySender_Group stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -250,8 +256,8 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
   /**
    * Test to validate the scenario gateway sender is started when one or more
    * sender members belongs to multiple groups
-   * 
    */
+  @Test
   public void testPauseGatewaySender_MultipleGroup() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -302,14 +308,14 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     vm7.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, false ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.PAUSE_GATEWAYSENDER + " --"
         + CliStrings.PAUSE_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.PAUSE_GATEWAYSENDER__GROUP + "=SenderGroup1,SenderGroup2";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testPauseGatewaySender_Group stringResult : " + strCmdResult
               + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -335,6 +341,7 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
         "ln", true, false ));
   }
 
+  @Test
   public void testResumeGatewaySender_ErrorConditions() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -360,7 +367,7 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testResumeGatewaySender_ErrorConditions stringResult : "
               + strCmdResult + ">>>>");
       assertEquals(Result.Status.ERROR, cmdResult.getStatus());
@@ -370,6 +377,7 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     }
   }
 
+  @Test
   public void testResumeGatewaySender() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -414,13 +422,13 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, true ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.RESUME_GATEWAYSENDER + " --"
         + CliStrings.RESUME_GATEWAYSENDER__ID + "=ln";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testResumeGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -446,6 +454,7 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
    * test to validate that the start gateway sender starts the gateway sender on
    * a member
    */
+  @Test
   public void testResumeGatewaySender_onMember() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -473,14 +482,14 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
         "ln", true, true ));
 
     final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.RESUME_GATEWAYSENDER + " --"
         + CliStrings.RESUME_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.RESUME_GATEWAYSENDER__MEMBER + "=" + vm1Member.getId();
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testResumeGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       assertTrue(strCmdResult.contains("is resumed on member"));
@@ -496,6 +505,7 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
    * test to validate that the start gateway sender starts the gateway sender on
    * a group of members
    */
+  @Test
   public void testResumeGatewaySender_Group() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -543,14 +553,14 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     vm5.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, true ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.RESUME_GATEWAYSENDER + " --"
         + CliStrings.RESUME_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.RESUME_GATEWAYSENDER__GROUP + "=SenderGroup1";
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testResumeGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -575,8 +585,8 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
   /**
    * Test to validate the scenario gateway sender is started when one or more
    * sender members belongs to multiple groups
-   * 
    */
+  @Test
   public void testResumeGatewaySender_MultipleGroup() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -644,7 +654,7 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     vm7.invoke(() -> WANCommandTestBase.verifySenderState(
         "ln", true, true ));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.RESUME_GATEWAYSENDER + " --"
         + CliStrings.RESUME_GATEWAYSENDER__ID + "=ln --"
         + CliStrings.RESUME_GATEWAYSENDER__GROUP
@@ -652,7 +662,7 @@ public class WanCommandPauseResumeDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testResumeGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       TabularResultData resultData = (TabularResultData) cmdResult

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
index 12d81d7..7293963 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandStatusDUnitTest.java
@@ -16,11 +16,16 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
 
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.management.cli.Result;
@@ -28,16 +33,14 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
-import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+@Category(DistributedTest.class)
 public class WanCommandStatusDUnitTest extends WANCommandTestBase{
   
   private static final long serialVersionUID = 1L;
 
-  public WanCommandStatusDUnitTest(String name) {
-    super(name);
-  }
-
+  @Test
   public void testGatewaySenderStatus(){
 
     Integer lnPort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -69,7 +72,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     vm5.invoke(() -> WANCommandTestBase.createSender(
         "ln_Parallel", 2, true, 100, 400, false, false, null, true));
 
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STATUS_GATEWAYSENDER + " --"
     + CliStrings.STATUS_GATEWAYSENDER__ID + "=ln_Serial";
     CommandResult cmdResult = executeCommand(command);
@@ -86,7 +89,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
       assertEquals(2, result_hosts.size());
       
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testGatewaySenderStatus : " + strCmdResult + ">>>>> ");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
@@ -102,7 +105,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     vm5.invoke(() -> WANCommandTestBase.startSender( "ln_Serial" ));
     vm5.invoke(() -> WANCommandTestBase.startSender( "ln_Parallel" ));
 
-    Wait.pause(10000);
+    pause(10000);
     command = CliStrings.STATUS_GATEWAYSENDER + " --"
     + CliStrings.STATUS_GATEWAYSENDER__ID + "=ln_Serial";
     cmdResult = executeCommand(command);
@@ -119,7 +122,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
       assertEquals(2, result_hosts.size());
       
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testGatewaySenderStatus : " + strCmdResult + ">>>>> ");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
@@ -127,7 +130,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     }
   }
 
-  
+  @Test
   public void testGatewaySenderStatus_OnMember(){
 
     Integer lnPort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -157,7 +160,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
 
     final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
     
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STATUS_GATEWAYSENDER + " --"
     + CliStrings.STATUS_GATEWAYSENDER__ID + "=ln_Serial --"
         + CliStrings.STATUS_GATEWAYSENDER__MEMBER + "=" + vm1Member.getId();
@@ -165,7 +168,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info("testGatewaySenderStatus_OnMember : " + strCmdResult + ">>>>> ");
+      getLogWriter().info("testGatewaySenderStatus_OnMember : " + strCmdResult + ">>>>> ");
       TabularResultData tableResultData =
           ((CompositeResultData)cmdResult.getResultData()).retrieveSection(CliStrings.SECTION_GATEWAY_SENDER_AVAILABLE).retrieveTable(CliStrings.TABLE_GATEWAY_SENDER);
       List<String> result_Status = tableResultData.retrieveAllValues(CliStrings.RESULT_STATUS);
@@ -184,7 +187,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     vm4.invoke(() -> WANCommandTestBase.startSender( "ln_Serial" ));
     vm4.invoke(() -> WANCommandTestBase.startSender( "ln_Parallel" ));
 
-    Wait.pause(10000);
+    pause(10000);
     command = CliStrings.STATUS_GATEWAYSENDER + " --"
     + CliStrings.STATUS_GATEWAYSENDER__ID + "=ln_Serial --"
         + CliStrings.STATUS_GATEWAYSENDER__MEMBER + "=" + vm1Member.getId();
@@ -197,7 +200,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
 //      assertEquals(1, result_Status.size());
 //      assertFalse(result_Status.contains(CliStrings.GATEWAY_NOT_RUNNING));
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info("testGatewaySenderStatus_OnMember : " + strCmdResult + ">>>>> ");
+      getLogWriter().info("testGatewaySenderStatus_OnMember : " + strCmdResult + ">>>>> ");
       TabularResultData tableResultData =
           ((CompositeResultData)cmdResult.getResultData()).retrieveSection(CliStrings.SECTION_GATEWAY_SENDER_AVAILABLE).retrieveTable(CliStrings.TABLE_GATEWAY_SENDER);
       List<String> result_Status = tableResultData.retrieveAllValues(CliStrings.RESULT_STATUS);
@@ -221,13 +224,14 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
       assertTrue(cmdResult != null);
       
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info("testGatewaySenderStatus_OnMember : " + strCmdResult + ">>>>> ");
+      getLogWriter().info("testGatewaySenderStatus_OnMember : " + strCmdResult + ">>>>> ");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testListGatewaySender failed as did not get CommandResult");
     }
   }
-  
+
+  @Test
   public void testGatewaySenderStatus_OnGroups(){
 
     Integer lnPort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -263,7 +267,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     
     final DistributedMember vm1Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
     
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STATUS_GATEWAYSENDER + " --"
     + CliStrings.STATUS_GATEWAYSENDER__ID + "=ln_Serial --" + CliStrings.STATUS_GATEWAYSENDER__GROUP + "=Serial_Sender";
     
@@ -282,7 +286,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
       
       
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testGatewaySenderStatus_OnGroups : " + strCmdResult + ">>>>> ");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
@@ -295,7 +299,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     vm4.invoke(() -> WANCommandTestBase.startSender( "ln_Serial" ));
     vm4.invoke(() -> WANCommandTestBase.startSender( "ln_Parallel" ));
 
-    Wait.pause(10000);
+    pause(10000);
     command = CliStrings.STATUS_GATEWAYSENDER + " --"
     + CliStrings.STATUS_GATEWAYSENDER__ID + "=ln_Serial --" + CliStrings.STATUS_GATEWAYSENDER__GROUP + "=Serial_Sender";
     
@@ -313,14 +317,15 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
       assertEquals(1, result_hosts.size());
       
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testGatewaySenderStatus_OnGroups : " + strCmdResult + ">>>>> ");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testListGatewaySender failed as did not get CommandResult");
     }
   }
-  
+
+  @Test
   public void testGatewayReceiverStatus(){
 
     Integer lnPort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -338,13 +343,13 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     vm4.invoke(() -> WANCommandTestBase.createAndStartReceiver( lnPort ));
     vm5.invoke(() -> WANCommandTestBase.createAndStartReceiver( lnPort ));
     
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STATUS_GATEWAYRECEIVER; 
     CommandResult cmdResult = executeCommand(command);
     
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testGatewayReceiverStatus : " + strCmdResult + ">>>>> ");
       
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -369,13 +374,13 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     vm4.invoke(() -> WANCommandTestBase.stopReceiver());
     vm5.invoke(() -> WANCommandTestBase.stopReceiver());
     
-    Wait.pause(10000);
+    pause(10000);
     
     command = CliStrings.STATUS_GATEWAYRECEIVER; 
     cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testGatewayReceiverStatus : " + strCmdResult + ">>>>> ");
       
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -396,7 +401,8 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
       fail("testGatewayReceiverStatus failed as did not get CommandResult");
     }
   }
-  
+
+  @Test
   public void testGatewayReceiverStatus_OnMember(){
 
     Integer lnPort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -416,7 +422,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     
     final DistributedMember vm3Member = (DistributedMember) vm3.invoke(() -> WANCommandTestBase.getMember());
     
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STATUS_GATEWAYRECEIVER+ " --"
     + CliStrings.STATUS_GATEWAYRECEIVER__MEMBER + "=" + vm3Member.getId();
     
@@ -424,7 +430,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info("testGatewayReceiverStatus : " + strCmdResult + ">>>>> ");
+      getLogWriter().info("testGatewayReceiverStatus : " + strCmdResult + ">>>>> ");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       //TabularResultData tableResultData = (TabularResultData) cmdResult.getResultData();
       //List<String> result_Status = tableResultData.retrieveAllValues(CliStrings.RESULT_STATUS);
@@ -443,7 +449,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     vm4.invoke(() -> WANCommandTestBase.stopReceiver());
     vm5.invoke(() -> WANCommandTestBase.stopReceiver());
     
-    Wait.pause(10000);
+    pause(10000);
     
     command = CliStrings.STATUS_GATEWAYRECEIVER+ " --"
     + CliStrings.STATUS_GATEWAYRECEIVER__MEMBER + "=" + vm3Member.getId();
@@ -451,7 +457,7 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testGatewayReceiverStatus : " + strCmdResult + ">>>>> ");
       
 //      TabularResultData tableResultData =
@@ -469,7 +475,8 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
       fail("testGatewayReceiverStatus failed as did not get CommandResult");
     }
   }
-  
+
+  @Test
   public void testGatewayReceiverStatus_OnGroups(){
 
     Integer lnPort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -488,14 +495,14 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     vm5.invoke(() -> WANCommandTestBase.createAndStartReceiverWithGroup( lnPort, "RG1"  ));
     vm6.invoke(() -> WANCommandTestBase.createAndStartReceiverWithGroup( lnPort, "RG2"  ));
     
-    Wait.pause(10000);
+    pause(10000);
     String command = CliStrings.STATUS_GATEWAYRECEIVER + " --"
         + CliStrings.STATUS_GATEWAYRECEIVER__GROUP + "=RG1";
     
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testGatewayReceiverStatus : " + strCmdResult + ">>>>> ");
       
       assertEquals(Result.Status.OK, cmdResult.getStatus());
@@ -515,13 +522,13 @@ public class WanCommandStatusDUnitTest extends WANCommandTestBase{
     vm4.invoke(() -> WANCommandTestBase.stopReceiver());
     vm5.invoke(() -> WANCommandTestBase.stopReceiver());
 
-    Wait.pause(10000);
+    pause(10000);
     command = CliStrings.STATUS_GATEWAYRECEIVER + " --"+ CliStrings.STATUS_GATEWAYRECEIVER__GROUP + "=RG1";
     
     cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testGatewayReceiverStatus_OnGroups : " + strCmdResult + ">>>>> ");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
index 43caf90..4ff9a73 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/management/internal/configuration/ClusterConfigurationDUnitTest.java
@@ -16,6 +16,27 @@
  */
 package com.gemstone.gemfire.management.internal.configuration;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -57,24 +78,10 @@ import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 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.DistributedTest;
 
-import org.apache.commons.io.FileUtils;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.Set;
-
+@Category(DistributedTest.class)
 public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
   private static final int TIMEOUT = 10000;
   private static final int INTERVAL = 500;
@@ -100,11 +107,7 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
   
   private static final long serialVersionUID = 1L;
 
-  public ClusterConfigurationDUnitTest(String name) {
-    super(name);
-  }
-  
-  
+  @Test
   public void testConfigDistribution() throws IOException {
     IgnoredException.addIgnoredException("could not get remote locator");
     try {
@@ -248,6 +251,7 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
    * @throws IOException
    * @since 8.1
    */
+  @Test
   public void testCreateExtensions() throws IOException {
     
     try {
@@ -321,6 +325,7 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
    * @throws IOException
    * @since 8.1
    */
+  @Test
   public void testDestroyExtensions() throws IOException {
     
     try {
@@ -382,7 +387,9 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
     }
   }
 
-  public void _testCreateDiskStore () throws IOException {
+  @Ignore("disabled for unknown reason")
+  @Test
+  public void testCreateDiskStore () throws IOException {
     try {
       Object[] result = setup();
       final int locatorPort = (Integer) result[0];
@@ -488,8 +495,10 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
     }
    
   }
-  
-  public void _testConfigurePDX() throws IOException {
+
+  @Ignore("disabled for unknown reason")
+  @Test
+  public void testConfigurePDX() throws IOException {
     try {
       Object[] result = setup();
       final int locatorPort = (Integer) result[0];
@@ -544,7 +553,8 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
      }
     }
   }
-  
+
+  @Test
   public void testClusterConfigDir() {
     disconnectAllFromDS();
     final int [] ports = AvailablePortHelper.getRandomAvailableTCPPorts(3);
@@ -605,7 +615,7 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
               return "Waiting for shared configuration to be started";
             }
           };
-          Wait.waitForCriterion(wc, TIMEOUT, INTERVAL, true);
+          waitForCriterion(wc, TIMEOUT, INTERVAL, true);
         } catch (IOException ioex) {
           fail("Unable to create a locator with a shared configuration");
         }
@@ -676,7 +686,7 @@ public class ClusterConfigurationDUnitTest extends CliCommandTestBase {
               return "Waiting for shared configuration to be started";
             }
           };
-          Wait.waitForCriterion(wc, TIMEOUT, INTERVAL, true);
+          waitForCriterion(wc, TIMEOUT, INTERVAL, true);
         } catch (IOException ioex) {
           fail("Unable to create a locator with a shared configuration");
         }


[16/32] incubator-geode git commit: GEODE-1201: Adding compileRuntimeLibs to geode-assembly for tests Amending bind address configuration for http-service-bind-address

Posted by ji...@apache.org.
GEODE-1201: Adding compileRuntimeLibs to geode-assembly for tests
Amending bind address configuration for http-service-bind-address


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

Branch: refs/heads/feature/GEODE-17-2
Commit: d1e48259894e60c54c588adba23a458f028e7145
Parents: a885ac1
Author: Udo Kohlmeyer <uk...@pivotal.io>
Authored: Mon Apr 11 07:57:21 2016 +1000
Committer: Udo Kohlmeyer <uk...@pivotal.io>
Committed: Thu Apr 14 10:07:06 2016 +1000

----------------------------------------------------------------------
 geode-assembly/build.gradle                     |  6 +++
 .../gemfire/management/internal/RestAgent.java  | 51 ++++++++++++++------
 .../web/swagger/config/RestApiPathProvider.java | 27 ++++++++++-
 3 files changed, 68 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1e48259/geode-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/geode-assembly/build.gradle b/geode-assembly/build.gradle
index b7d05e2..8f14fc1 100755
--- a/geode-assembly/build.gradle
+++ b/geode-assembly/build.gradle
@@ -81,6 +81,7 @@ sourceSets {
 
 test {
   // test from the actual classpath not the gradle classpath
+  dependsOn copyRuntimeLibs
   dependsOn installDist
   // @TODO: this doesn't seem to be working need to get basename first.
   classpath += files "$buildDir/install/apache-geode/lib/geode-dependencies.jar"
@@ -112,6 +113,11 @@ task defaultCacheConfig(type: JavaExec, dependsOn: classes) {
   }
 }
 
+task copyRuntimeLibs(type: Copy) {
+  into "lib"
+  from configurations.testRuntime - configurations.runtime
+}
+
 // This closure sets the gemfire classpath.  If we add another jar to the classpath it must
 // be included in the filter logic below.
 def cp = {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1e48259/geode-core/src/main/java/com/gemstone/gemfire/management/internal/RestAgent.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/RestAgent.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/RestAgent.java
index a91df05..526bd2d 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/RestAgent.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/RestAgent.java
@@ -17,26 +17,25 @@
 
 package com.gemstone.gemfire.management.internal;
 
-import org.apache.logging.log4j.Logger;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.ServerConnector;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.DataPolicy;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.GemFireVersion;
+import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.management.ManagementService;
+import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.Logger;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+
+import java.net.UnknownHostException;
 
 /**
  * Agent implementation that controls the HTTP server end points used for REST
  * clients to connect gemfire data node.
- * 
+ * <p>
  * The RestAgent is used to start http service in embedded mode on any non
  * manager data node with developer REST APIs service enabled.
  *
@@ -109,8 +108,9 @@ public class RestAgent {
   public void startHttpService() {
     // TODO: add a check that will make sure that we start HTTP service on
     // non-manager data node
+    String httpServiceBindAddress = getBindAddressForHttpService();
     logger.info("Attempting to start HTTP service on port ({}) at bind-address ({})...",
-        this.config.getHttpServicePort(), this.config.getHttpServiceBindAddress());
+        this.config.getHttpServicePort(), httpServiceBindAddress);
 
     // Find the developer REST WAR file
     final String gemfireAPIWar = agentUtil.findWarLocation("geode-web-api");
@@ -124,10 +124,9 @@ public class RestAgent {
         logger.warn("Detected presence of catalina system properties. HTTP service will not be started. To enable the GemFire Developer REST API, please deploy the /geode-web-api WAR file in your application server."); 
       } else if (agentUtil.isWebApplicationAvailable(gemfireAPIWar)) {
 
-        final String bindAddress = this.config.getHttpServiceBindAddress();
         final int port = this.config.getHttpServicePort();
 
-        this.httpServer = JettyHelper.initJetty(bindAddress, port,
+        this.httpServer = JettyHelper.initJetty(httpServiceBindAddress, port,
             this.config.getHttpServiceSSLEnabled(),
             this.config.getHttpServiceSSLRequireAuthentication(),
             this.config.getHttpServiceSSLProtocols(), this.config.getHttpServiceSSLCiphers(),
@@ -136,8 +135,8 @@ public class RestAgent {
         this.httpServer = JettyHelper.addWebApplication(httpServer, "/gemfire-api", gemfireAPIWar);
 
         if (logger.isDebugEnabled()) {
-          logger.debug("Starting HTTP embedded server on port ({}) at bind-address ({})...",
-              ((ServerConnector) this.httpServer.getConnectors()[0]).getPort(), bindAddress);
+          logger.info("Starting HTTP embedded server on port ({}) at bind-address ({})...",
+              ((ServerConnector) this.httpServer.getConnectors()[0]).getPort(), httpServiceBindAddress);
         }
 
         this.httpServer = JettyHelper.startJetty(this.httpServer);
@@ -151,6 +150,28 @@ public class RestAgent {
     }
   }
 
+  private String getBindAddressForHttpService() {
+    java.lang.String bindAddress = this.config.getHttpServiceBindAddress();
+    if (StringUtils.isBlank(bindAddress)) {
+      if (StringUtils.isBlank(this.config.getServerBindAddress())) {
+        if (StringUtils.isBlank(this.config.getBindAddress())) {
+          try {
+            bindAddress = SocketCreator.getLocalHost().getHostAddress();
+            logger.info("RestAgent.getBindAddressForHttpService.localhost: " + SocketCreator.getLocalHost().getHostAddress());
+          } catch (UnknownHostException e) {
+            logger.error("LocalHost could not be found.", e);
+            return bindAddress;
+          }
+        } else {
+          bindAddress = this.config.getBindAddress();
+        }
+      } else {
+        bindAddress = this.config.getServerBindAddress();
+      }
+    }
+    return bindAddress;
+  }
+
   private void stopHttpService() {
     if (this.httpServer != null) {
       logger.info("Stopping the HTTP service...");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d1e48259/geode-web-api/src/main/java/com/gemstone/gemfire/rest/internal/web/swagger/config/RestApiPathProvider.java
----------------------------------------------------------------------
diff --git a/geode-web-api/src/main/java/com/gemstone/gemfire/rest/internal/web/swagger/config/RestApiPathProvider.java b/geode-web-api/src/main/java/com/gemstone/gemfire/rest/internal/web/swagger/config/RestApiPathProvider.java
index a8921d7..47316ed 100644
--- a/geode-web-api/src/main/java/com/gemstone/gemfire/rest/internal/web/swagger/config/RestApiPathProvider.java
+++ b/geode-web-api/src/main/java/com/gemstone/gemfire/rest/internal/web/swagger/config/RestApiPathProvider.java
@@ -18,8 +18,10 @@ package com.gemstone.gemfire.rest.internal.web.swagger.config;
 
 import javax.servlet.ServletContext;
 
+import com.gemstone.gemfire.admin.internal.InetAddressUtil;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.lang.StringUtils;
 import com.mangofactory.swagger.core.SwaggerPathProvider;
 
@@ -27,6 +29,8 @@ import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.util.Assert;
 import org.springframework.web.util.UriComponentsBuilder;
 
+import java.net.UnknownHostException;
+
 @SuppressWarnings("unused")
 public class RestApiPathProvider implements SwaggerPathProvider {
 
@@ -44,7 +48,28 @@ public class RestApiPathProvider implements SwaggerPathProvider {
     DistributionConfig config = InternalDistributedSystem.getAnyInstance().getConfig();
     String scheme = config.getHttpServiceSSLEnabled() ? "https" : "http";
 
-    this.docsLocation = scheme + "://" + config.getHttpServiceBindAddress() + ":" + config.getHttpServicePort();
+    this.docsLocation = scheme + "://" + getBindAddressForHttpService() + ":" + config.getHttpServicePort();
+  }
+
+  private String getBindAddressForHttpService() {
+    DistributionConfig config = InternalDistributedSystem.getAnyInstance().getConfig();
+    java.lang.String bindAddress = config.getHttpServiceBindAddress();
+    if (org.apache.commons.lang.StringUtils.isBlank(bindAddress)) {
+      if (org.apache.commons.lang.StringUtils.isBlank(config.getServerBindAddress())) {
+        if (org.apache.commons.lang.StringUtils.isBlank(config.getBindAddress())) {
+          try {
+          bindAddress = SocketCreator.getLocalHost().getHostAddress();
+          } catch (UnknownHostException e) {
+            e.printStackTrace();
+          }
+        } else {
+          bindAddress = config.getBindAddress();
+        }
+      } else {
+        bindAddress = config.getServerBindAddress();
+      }
+    }
+    return bindAddress;
   }
 
   @Override


[22/32] incubator-geode git commit: GEODE-1162: convert all CLI command DUnit tests to JUnit 4

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java
index d44a479..1c6f141 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsDUnitTest.java
@@ -16,6 +16,20 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.IgnoredException.*;
+import static com.gemstone.gemfire.test.dunit.Invoke.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Properties;
+
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -32,41 +46,29 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData;
 import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData.SectionResultData;
+import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
+import com.gemstone.gemfire.management.internal.cli.result.ResultData;
+import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
-import com.gemstone.gemfire.test.dunit.Invoke;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.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.management.internal.cli.result.ResultBuilder;
-import com.gemstone.gemfire.management.internal.cli.result.ResultData;
-import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
-
-import org.junit.Ignore;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Properties;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Dunit class for testing gemfire function commands : GC, Shutdown
- *
  */
+@Category({ DistributedTest.class, FlakyTest.class }) // see GEODE-1034
 public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
   private static String cachedLogLevel;
 
-  public MiscellaneousCommandsDUnitTest(String name) {
-    super(name);
-  }
-
   @Override
   protected final void preTearDownCliCommandTestBase() throws Exception {
-    Invoke.invokeInEveryVM(new SerializableRunnable("reset log level") {
+    invokeInEveryVM(new SerializableRunnable("reset log level") {
       public void run() {
         if (cachedLogLevel != null) {
           System.setProperty("gemfire.log-level", cachedLogLevel);
@@ -76,6 +78,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     });
   }
 
+  @Test
   public void testGCForGroup() {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
@@ -86,7 +89,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     cmdResult.resetToFirstLine();
     if (cmdResult != null) {
       String cmdResultStr = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testGCForGroup cmdResultStr=" + cmdResultStr + "; cmdResult=" + cmdResult);
+      getLogWriter().info("testGCForGroup cmdResultStr=" + cmdResultStr + "; cmdResult=" + cmdResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       if (cmdResult.getType().equals(ResultData.TYPE_TABULAR)) {
         TabularResultData table = (TabularResultData) cmdResult.getResultData();
@@ -101,10 +104,11 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
   }
 
   public static String getMemberId() {
-    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    Cache cache = new GemfireDataCommandsDUnitTest().getCache();
     return cache.getDistributedSystem().getDistributedMember().getId();
   }
 
+  @Test
   public void testGCForMemberID() {
     createDefaultSetup(null);
     final VM vm1 = Host.getHost(0).getVM(1);
@@ -114,7 +118,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     cmdResult.resetToFirstLine();
     if (cmdResult != null) {
       String cmdResultStr = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testGCForMemberID cmdResultStr=" + cmdResultStr);
+      getLogWriter().info("testGCForMemberID cmdResultStr=" + cmdResultStr);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       if (cmdResult.getType().equals(ResultData.TYPE_TABULAR)) {
         TabularResultData table = (TabularResultData) cmdResult.getResultData();
@@ -128,6 +132,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testShowLogDefault() throws IOException {
     Properties props = new Properties();
     try {
@@ -140,7 +145,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
       if (cmdResult != null) {
         String log = commandResultToString(cmdResult);
         assertNotNull(log);
-        LogWriterUtils.getLogWriter().info("Show Log is" + log);
+        getLogWriter().info("Show Log is" + log);
         assertEquals(Result.Status.OK, cmdResult.getStatus());
       } else {
         fail("testShowLog failed as did not get CommandResult");
@@ -150,6 +155,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testShowLogNumLines() {
     Properties props = new Properties();
     props.setProperty("log-file", "testShowLogNumLines.log");
@@ -162,7 +168,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
       if (cmdResult != null) {
         String log = commandResultToString(cmdResult);
         assertNotNull(log);
-        LogWriterUtils.getLogWriter().info("Show Log is" + log);
+        getLogWriter().info("Show Log is" + log);
         assertEquals(Result.Status.OK, cmdResult.getStatus());
       } else {
         fail("testShowLog failed as did not get CommandResult");
@@ -172,6 +178,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testGCForEntireCluster() {
     setupForGC();
     String command = "gc";
@@ -179,7 +186,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     cmdResult.resetToFirstLine();
     if (cmdResult != null) {
       String cmdResultStr = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testGCForEntireCluster cmdResultStr=" + cmdResultStr + "; cmdResult=" + cmdResult);
+      getLogWriter().info("testGCForEntireCluster cmdResultStr=" + cmdResultStr + "; cmdResult=" + cmdResult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
       if (cmdResult.getType().equals(ResultData.TYPE_TABULAR)) {
         TabularResultData table = (TabularResultData) cmdResult.getResultData();
@@ -224,9 +231,10 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     });
   }
 
+  @Test
   public void testShutDownWithoutTimeout() {
 
-    IgnoredException.addIgnoredException("EntryDestroyedException");
+    addIgnoredException("EntryDestroyedException");
 
     setupForShutDown();
     ThreadUtils.sleep(2500);
@@ -236,7 +244,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
 
     if (cmdResult != null) {
       String cmdResultStr = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testShutDownWithoutTimeout cmdResultStr=" + cmdResultStr);
+      getLogWriter().info("testShutDownWithoutTimeout cmdResultStr=" + cmdResultStr);
     }
 
     verifyShutDown();
@@ -245,7 +253,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
 
     // Need for the Gfsh HTTP enablement during shutdown to properly assess the
     // state of the connection.
-    Wait.waitForCriterion(new WaitCriterion() {
+    waitForCriterion(new WaitCriterion() {
       public boolean done() {
         return !defaultShell.isConnectedAndReady();
       }
@@ -259,18 +267,19 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
   }
 
   @Ignore("Disabled for 52350")
-  public void DISABLED_testShutDownWithTimeout() {
+  @Test
+  public void testShutDownWithTimeout() {
     setupForShutDown();
     ThreadUtils.sleep(2500);
 
-    IgnoredException.addIgnoredException("EntryDestroyedException");
+    addIgnoredException("EntryDestroyedException");
 
     String command = "shutdown --time-out=15";
     CommandResult cmdResult = executeCommand(command);
 
     if (cmdResult != null) {
       String cmdResultStr = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testShutDownWithTIMEOUT cmdResultStr=" + cmdResultStr);
+      getLogWriter().info("testShutDownWithTIMEOUT cmdResultStr=" + cmdResultStr);
     }
 
     verifyShutDown();
@@ -278,7 +287,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     final HeadlessGfsh defaultShell = getDefaultShell();
 
     // Need for the Gfsh HTTP enablement during shutdown to properly assess the state of the connection.
-    Wait.waitForCriterion(new WaitCriterion() {
+    waitForCriterion(new WaitCriterion() {
       public boolean done() {
         return !defaultShell.isConnectedAndReady();
       }
@@ -291,6 +300,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     assertFalse(defaultShell.isConnectedAndReady());
   }
 
+  @Test
   public void testShutDownForTIMEOUT() {
     setupForShutDown();
     ThreadUtils.sleep(2500);
@@ -307,7 +317,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
 
     if (cmdResult != null) {
       String cmdResultStr = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testShutDownForTIMEOUT cmdResultStr = " + cmdResultStr);
+      getLogWriter().info("testShutDownForTIMEOUT cmdResultStr = " + cmdResultStr);
       CommandResult result = (CommandResult) ResultBuilder.createInfoResult(CliStrings.SHUTDOWN_TIMEDOUT);
       String expectedResult = commandResultToString(result);
       assertEquals(expectedResult, cmdResultStr);
@@ -387,12 +397,13 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
         return "Wait for gfsh to get disconnected from Manager.";
       }
     };
-    Wait.waitForCriterion(waitCriterion, 5000, 200, true);
+    waitForCriterion(waitCriterion, 5000, 200, true);
 
     assertTrue(Boolean.FALSE.equals(vm1.invoke(connectedChecker)));
     assertTrue(Boolean.FALSE.equals(vm0.invoke(connectedChecker)));
   }
 
+  @Test
   public void testChangeLogLevelForMembers() {
     final VM vm0 = Host.getHost(0).getVM(0);
     final VM vm1 = Host.getHost(0).getVM(1);
@@ -418,7 +429,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     String commandString = CliStrings.CHANGE_LOGLEVEL + " --" + CliStrings.CHANGE_LOGLEVEL__LOGLEVEL + "=finer" + " --" + CliStrings.CHANGE_LOGLEVEL__MEMBER + "=" + serverName1 + "," + serverName2;
 
     CommandResult commandResult = executeCommand(commandString);
-    LogWriterUtils.getLogWriter().info("testChangeLogLevel commandResult=" + commandResult);
+    getLogWriter().info("testChangeLogLevel commandResult=" + commandResult);
     assertTrue(Status.OK.equals(commandResult.getStatus()));
     CompositeResultData resultData = (CompositeResultData) commandResult.getResultData();
     SectionResultData section = resultData.retrieveSection("section");
@@ -437,6 +448,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     assertTrue(status.contains("true"));
   }
 
+  @Test
   public void testChangeLogLevelForGrps() {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
@@ -474,7 +486,7 @@ public class MiscellaneousCommandsDUnitTest extends CliCommandTestBase {
     String commandString = CliStrings.CHANGE_LOGLEVEL + " --" + CliStrings.CHANGE_LOGLEVEL__LOGLEVEL + "=finer" + " --" + CliStrings.CHANGE_LOGLEVEL__GROUPS + "=" + grp1 + "," + grp2;
 
     CommandResult commandResult = executeCommand(commandString);
-    LogWriterUtils.getLogWriter().info("testChangeLogLevelForGrps commandResult=" + commandResult);
+    getLogWriter().info("testChangeLogLevelForGrps commandResult=" + commandResult);
 
     assertTrue(Status.OK.equals(commandResult.getStatus()));
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java
index b9178b6..60c5ded 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart1DUnitTest.java
@@ -16,6 +16,17 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
@@ -25,30 +36,20 @@ import com.gemstone.gemfire.internal.logging.LogWriterImpl;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 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 java.io.File;
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Dunit class for testing gemfire function commands : export logs
- *
  */
-
+@Category(DistributedTest.class)
 public class MiscellaneousCommandsExportLogsPart1DUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public MiscellaneousCommandsExportLogsPart1DUnitTest(String name) {
-    super(name);
-  }
-
   public static String getMemberId() {
-    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    Cache cache = new GemfireDataCommandsDUnitTest().getCache();
     return cache.getDistributedSystem().getDistributedMember().getId();
   }
 
@@ -77,6 +78,7 @@ public class MiscellaneousCommandsExportLogsPart1DUnitTest extends CliCommandTes
     return ("_" + formattedStartDate);
   }
 
+  @Test
   public void testExportLogs() throws IOException {
     Date startDate = new Date(System.currentTimeMillis() - 2 * 60 * 1000);
     SimpleDateFormat sf = new SimpleDateFormat("yyyy/MM/dd");
@@ -95,11 +97,11 @@ public class MiscellaneousCommandsExportLogsPart1DUnitTest extends CliCommandTes
     Result cmdResult = misc.exportLogsPreprocessing("./testExportLogs" + dir, null, null, logLevel, false, false, start,
         end, 1);
 
-    LogWriterUtils.getLogWriter().info("testExportLogs command result =" + cmdResult);
+    getLogWriter().info("testExportLogs command result =" + cmdResult);
 
     if (cmdResult != null) {
       String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
-      LogWriterUtils.getLogWriter().info("testExportLogs cmdStringRsult=" + cmdStringRsult);
+      getLogWriter().info("testExportLogs cmdStringRsult=" + cmdStringRsult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testExportLogs failed as did not get CommandResult");
@@ -107,6 +109,7 @@ public class MiscellaneousCommandsExportLogsPart1DUnitTest extends CliCommandTes
     FileUtil.delete(new File("./testExportLogs" + dir));
   }
 
+  @Test
   public void testExportLogsForMerge() throws IOException {
     setupForExportLogs();
     Date startDate = new Date(System.currentTimeMillis() - 2 * 60 * 1000);
@@ -124,11 +127,11 @@ public class MiscellaneousCommandsExportLogsPart1DUnitTest extends CliCommandTes
 
     Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForMerge" + dir, null, null, logLevel, false, true,
         start, end, 1);
-    LogWriterUtils.getLogWriter().info("testExportLogsForMerge command=" + cmdResult);
+    getLogWriter().info("testExportLogsForMerge command=" + cmdResult);
 
     if (cmdResult != null) {
       String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
-      LogWriterUtils.getLogWriter().info("testExportLogsForMerge cmdStringRsult=" + cmdStringRsult);
+      getLogWriter().info("testExportLogsForMerge cmdStringRsult=" + cmdStringRsult);
 
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java
index 678af5a..fbe156a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart2DUnitTest.java
@@ -16,6 +16,17 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
@@ -25,30 +36,20 @@ import com.gemstone.gemfire.internal.logging.LogWriterImpl;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 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 java.io.File;
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Dunit class for testing gemfire function commands : export logs
- *
  */
-
+@Category(DistributedTest.class)
 public class MiscellaneousCommandsExportLogsPart2DUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public MiscellaneousCommandsExportLogsPart2DUnitTest(String name) {
-    super(name);
-  }
-
   public static String getMemberId() {
-    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    Cache cache = new GemfireDataCommandsDUnitTest().getCache();
     return cache.getDistributedSystem().getDistributedMember().getId();
   }
 
@@ -77,6 +78,7 @@ public class MiscellaneousCommandsExportLogsPart2DUnitTest extends CliCommandTes
     return ("_" + formattedStartDate);
   }
 
+  @Test
   public void testExportLogsForLogLevel() throws IOException {
     setupForExportLogs();
 
@@ -96,11 +98,11 @@ public class MiscellaneousCommandsExportLogsPart2DUnitTest extends CliCommandTes
     Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForLogLevel" + dir, null, null, logLevel, false,
         false, start, end, 1);
 
-    LogWriterUtils.getLogWriter().info("testExportLogsForLogLevel command=" + cmdResult);
+    getLogWriter().info("testExportLogsForLogLevel command=" + cmdResult);
 
     if (cmdResult != null) {
       String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
-      LogWriterUtils.getLogWriter().info("testExportLogsForLogLevel cmdStringRsult=" + cmdStringRsult);
+      getLogWriter().info("testExportLogsForLogLevel cmdStringRsult=" + cmdStringRsult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testExportLogsForLogLevel failed as did not get CommandResult");
@@ -108,7 +110,7 @@ public class MiscellaneousCommandsExportLogsPart2DUnitTest extends CliCommandTes
     FileUtil.delete(new File("testExportLogsForLogLevel" + dir));
   }
 
-
+  @Test
   public void testExportLogsForLogLevelWithUPTOLOGLEVEL() throws IOException {
     setupForExportLogs();
 
@@ -128,11 +130,11 @@ public class MiscellaneousCommandsExportLogsPart2DUnitTest extends CliCommandTes
     Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForLogLevelWithUPTOLOGLEVEL" + dir, null, null,
         logLevel, true, false, start, end, 1);
 
-    LogWriterUtils.getLogWriter().info("testExportLogsForLogLevelWithUPTOLOGLEVEL command=" + cmdResult);
+    getLogWriter().info("testExportLogsForLogLevelWithUPTOLOGLEVEL command=" + cmdResult);
 
     if (cmdResult != null) {
       String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
-      LogWriterUtils.getLogWriter().info("testExportLogsForLogLevelWithUPTOLOGLEVEL cmdStringRsult=" + cmdStringRsult);
+      getLogWriter().info("testExportLogsForLogLevelWithUPTOLOGLEVEL cmdStringRsult=" + cmdStringRsult);
 
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java
index 361a8d1..5c159e8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart3DUnitTest.java
@@ -16,6 +16,18 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Properties;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
@@ -26,31 +38,20 @@ import com.gemstone.gemfire.internal.logging.LogWriterImpl;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 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 java.io.File;
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Properties;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Dunit class for testing gemfire function commands : export logs
- *
  */
-
+@Category(DistributedTest.class)
 public class MiscellaneousCommandsExportLogsPart3DUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public MiscellaneousCommandsExportLogsPart3DUnitTest(String name) {
-    super(name);
-  }
-
   public static String getMemberId() {
-    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    Cache cache = new GemfireDataCommandsDUnitTest().getCache();
     return cache.getDistributedSystem().getDistributedMember().getId();
   }
 
@@ -79,6 +80,7 @@ public class MiscellaneousCommandsExportLogsPart3DUnitTest extends CliCommandTes
     return ("_" + formattedStartDate);
   }
 
+  @Test
   public void testExportLogsForGroup() throws IOException {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Manager");
@@ -103,10 +105,10 @@ public class MiscellaneousCommandsExportLogsPart3DUnitTest extends CliCommandTes
     Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForGroup" + dir, groups, null, logLevel, false,
         false, start, end, 1);
 
-    LogWriterUtils.getLogWriter().info("testExportLogsForGroup command result =" + cmdResult);
+    getLogWriter().info("testExportLogsForGroup command result =" + cmdResult);
     if (cmdResult != null) {
       String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
-      LogWriterUtils.getLogWriter().info("testExportLogsForGroup cmdStringRsult=" + cmdStringRsult);
+      getLogWriter().info("testExportLogsForGroup cmdStringRsult=" + cmdStringRsult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testExportLogsForGroup failed as did not get CommandResult");
@@ -114,6 +116,7 @@ public class MiscellaneousCommandsExportLogsPart3DUnitTest extends CliCommandTes
     FileUtil.delete(new File("testExportLogsForGroup" + dir));
   }
 
+  @Test
   public void testExportLogsForMember() throws IOException {
     createDefaultSetup(null);
 
@@ -136,11 +139,11 @@ public class MiscellaneousCommandsExportLogsPart3DUnitTest extends CliCommandTes
     Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForMember" + dir, null, vm1MemberId, logLevel,
         false, false, start, end, 1);
 
-    LogWriterUtils.getLogWriter().info("testExportLogsForMember command result =" + cmdResult);
+    getLogWriter().info("testExportLogsForMember command result =" + cmdResult);
 
     if (cmdResult != null) {
       String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
-      LogWriterUtils.getLogWriter().info("testExportLogsForMember cmdStringRsult=" + cmdStringRsult);
+      getLogWriter().info("testExportLogsForMember cmdStringRsult=" + cmdStringRsult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testExportLogsForMember failed as did not get CommandResult");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java
index 360ba08..cf19659 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommandsExportLogsPart4DUnitTest.java
@@ -16,6 +16,17 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
@@ -25,29 +36,20 @@ import com.gemstone.gemfire.internal.logging.LogWriterImpl;
 import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 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 java.io.File;
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Dunit class for testing gemfire function commands : export logs
- *
  */
+@Category(DistributedTest.class)
 public class MiscellaneousCommandsExportLogsPart4DUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public MiscellaneousCommandsExportLogsPart4DUnitTest(String name) {
-    super(name);
-  }
-
   public static String getMemberId() {
-    Cache cache = new GemfireDataCommandsDUnitTest("test").getCache();
+    Cache cache = new GemfireDataCommandsDUnitTest().getCache();
     return cache.getDistributedSystem().getDistributedMember().getId();
   }
 
@@ -76,6 +78,7 @@ public class MiscellaneousCommandsExportLogsPart4DUnitTest extends CliCommandTes
     return ("_" + formattedStartDate);
   }
 
+  @Test
   public void testExportLogsForTimeRange1() throws IOException {
     setupForExportLogs();
     Date startDate = new Date(System.currentTimeMillis() - 1 * 60 * 1000);
@@ -94,11 +97,11 @@ public class MiscellaneousCommandsExportLogsPart4DUnitTest extends CliCommandTes
     Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForTimeRange1" + dir, null, null, logLevel, false,
         false, start, end, 1);
 
-    LogWriterUtils.getLogWriter().info("testExportLogsForTimeRange1 command result =" + cmdResult);
+    getLogWriter().info("testExportLogsForTimeRange1 command result =" + cmdResult);
 
     if (cmdResult != null) {
       String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
-      LogWriterUtils.getLogWriter().info("testExportLogsForTimeRange1 cmdStringRsult=" + cmdStringRsult);
+      getLogWriter().info("testExportLogsForTimeRange1 cmdStringRsult=" + cmdStringRsult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testExportLogsForTimeRange1 failed as did not get CommandResult");
@@ -106,6 +109,7 @@ public class MiscellaneousCommandsExportLogsPart4DUnitTest extends CliCommandTes
     FileUtil.delete(new File("testExportLogsForTimeRange1" + dir));
   }
 
+  @Test
   public void testExportLogsForTimeRangeForOnlyStartTime() throws IOException {
     setupForExportLogs();
     Date date = new Date();
@@ -122,11 +126,11 @@ public class MiscellaneousCommandsExportLogsPart4DUnitTest extends CliCommandTes
     Result cmdResult = misc.exportLogsPreprocessing("./testExportLogsForTimeRangeForOnlyStartTime" + dir, null, null,
         logLevel, false, false, s, null, 1);
 
-    LogWriterUtils.getLogWriter().info("testExportLogsForTimeRangeForOnlyStartTime command result =" + cmdResult);
+    getLogWriter().info("testExportLogsForTimeRangeForOnlyStartTime command result =" + cmdResult);
 
     if (cmdResult != null) {
       String cmdStringRsult = commandResultToString((CommandResult) cmdResult);
-      LogWriterUtils.getLogWriter().info("testExportLogsForTimeRangeForOnlyStartTime cmdStringRsult=" + cmdStringRsult);
+      getLogWriter().info("testExportLogsForTimeRangeForOnlyStartTime cmdStringRsult=" + cmdStringRsult);
       assertEquals(Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testExportLogsForTimeRangeForOnlyStartTime failed as did not get CommandResult");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
index c4d1f93..f61ff57 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
@@ -16,6 +16,21 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.distributed.Locator;
@@ -29,41 +44,30 @@ import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.Assert;
 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.dunit.WaitCriterion;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.CopyOnWriteArrayList;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * A distributed test suite of test cases for testing the queue commands that are part of Gfsh.
  *
  * @since 8.0
  */
+@Category(DistributedTest.class)
 public class QueueCommandsDUnitTest extends CliCommandTestBase {
+
   private static final long serialVersionUID = 1L;
 
   final List<String> filesToBeDeleted = new CopyOnWriteArrayList<String>();
 
-  public QueueCommandsDUnitTest(final String testName) {
-    super(testName);
-  }
-
   @Override
   public final void preSetUp() throws Exception {
     disconnectAllFromDS();
   }
 
+  @Test
   public void testAsyncEventQueue() throws IOException {
     final String queue1Name = "testAsyncEventQueue1";
     final String queue2Name = "testAsyncEventQueue2";
@@ -236,6 +240,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
   /**
    * Asserts that creating async event queues correctly updates the shared configuration.
    */
+  @Test
   public void testCreateUpdatesSharedConfig() throws IOException {
     disconnectAllFromDS();
 
@@ -269,7 +274,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
               return "Waiting for shared configuration to be started";
             }
           };
-          Wait.waitForCriterion(wc, 5000, 500, true);
+          waitForCriterion(wc, 5000, 500, true);
         } catch (IOException ioex) {
           fail("Unable to create a locator with a shared configuration");
         }
@@ -336,7 +341,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
           xmlFromConfig = sharedConfig.getConfiguration(groupName).getCacheXmlContent();
           assertTrue(xmlFromConfig.contains(queueName));
         } catch (Exception e) {
-          Assert.fail("Error occurred in cluster configuration service", e);
+          fail("Error occurred in cluster configuration service", e);
         }
       }
     });
@@ -377,7 +382,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
           executeCommand("undeploy --jar=" + fileToDelete.getName());
         }
       } catch (IOException e) {
-        LogWriterUtils.getLogWriter().error("Unable to delete file", e);
+        getLogWriter().error("Unable to delete file", e);
       }
     }
     this.filesToBeDeleted.clear();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
index 5805853..d3e004b 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/SharedConfigurationCommandsDUnitTest.java
@@ -16,6 +16,21 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.DistributedMember;
@@ -34,42 +49,28 @@ import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 import com.gemstone.gemfire.management.internal.configuration.SharedConfigurationDUnitTest;
 import com.gemstone.gemfire.management.internal.configuration.domain.Configuration;
-import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 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 org.apache.commons.io.FileUtils;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Properties;
-import java.util.Set;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /***
  * DUnit test to test export and import of shared configuration.
- *
  */
+@Category(DistributedTest.class)
+@SuppressWarnings("unchecked")
 public class SharedConfigurationCommandsDUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
   private static final int TIMEOUT = 10000;
   private static final int INTERVAL = 500;
 
-  public SharedConfigurationCommandsDUnitTest(String name) {
-    super(name);
-  }
-
   File newDeployableJarFile = new File("DeployCommandsDUnit1.jar");
   private transient ClassBuilder classBuilder = new ClassBuilder();
 
-  @SuppressWarnings("unchecked")
+  @Test
   public void testExportImportSharedConfiguration() {
     disconnectAllFromDS();
 
@@ -134,7 +135,7 @@ public class SharedConfigurationCommandsDUnitTest extends CliCommandTestBase {
               return "Waiting for shared configuration to be started";
             }
           };
-          Wait.waitForCriterion(wc, TIMEOUT, INTERVAL, true);
+          waitForCriterion(wc, TIMEOUT, INTERVAL, true);
         } catch (IOException ioex) {
           fail("Unable to create a locator with a shared configuration");
         }
@@ -213,23 +214,23 @@ public class SharedConfigurationCommandsDUnitTest extends CliCommandTestBase {
     cmdResult = executeCommand(commandStringBuilder.getCommandString());
     String resultString = commandResultToString(cmdResult);
 
-    LogWriterUtils.getLogWriter().info("#SB Result\n");
-    LogWriterUtils.getLogWriter().info(resultString);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultString);
     assertEquals(true, cmdResult.getStatus().equals(Status.OK));
 
     commandStringBuilder = new CommandStringBuilder(CliStrings.STATUS_SHARED_CONFIG);
     cmdResult = executeCommand(commandStringBuilder.getCommandString());
     resultString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("#SB Result\n");
-    LogWriterUtils.getLogWriter().info(resultString);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultString);
     assertEquals(Status.OK, cmdResult.getStatus());
 
     commandStringBuilder = new CommandStringBuilder(CliStrings.EXPORT_SHARED_CONFIG);
     commandStringBuilder.addOption(CliStrings.EXPORT_SHARED_CONFIG__FILE, sharedConfigZipFileName);
     cmdResult = executeCommand(commandStringBuilder.getCommandString());
     resultString = commandResultToString(cmdResult);
-    LogWriterUtils.getLogWriter().info("#SB Result\n");
-    LogWriterUtils.getLogWriter().info(resultString);
+    getLogWriter().info("#SB Result\n");
+    getLogWriter().info(resultString);
     assertEquals(Status.OK, cmdResult.getStatus());
 
     //Import into a running system should fail
@@ -300,7 +301,7 @@ public class SharedConfigurationCommandsDUnitTest extends CliCommandTestBase {
               return "Waiting for shared configuration to be started";
             }
           };
-          Wait.waitForCriterion(wc, 5000, 500, true);
+          waitForCriterion(wc, 5000, 500, true);
 
           SharedConfiguration sc = locator.getSharedConfiguration();
           assertNotNull(sc);
@@ -320,7 +321,7 @@ public class SharedConfigurationCommandsDUnitTest extends CliCommandTestBase {
         } catch (IOException ioex) {
           fail("Unable to create a locator with a shared configuration");
         } catch (Exception e) {
-          Assert.fail("Error occurred in cluster configuration service", e);
+          fail("Error occurred in cluster configuration service", e);
         }
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
index ee6fcac..f5aa506 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommandsDUnitTest.java
@@ -16,6 +16,15 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
+import java.io.File;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
 import com.gemstone.gemfire.distributed.LocatorLauncher;
 import com.gemstone.gemfire.distributed.LocatorLauncher.LocatorState;
@@ -28,21 +37,13 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.shell.Gfsh;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
-
-import org.junit.Before;
-
-import java.io.File;
-import java.util.concurrent.TimeUnit;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+@Category(DistributedTest.class)
 public class ShellCommandsDUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public ShellCommandsDUnitTest(String name) {
-    super(name);
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     getDefaultShell();
@@ -53,6 +54,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
         "localhost[" + locatorPort + "]").toString());
   }
 
+  @Test
   public void testConnectToLocatorBecomesManager() {
     final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
 
@@ -98,13 +100,14 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testEchoWithVariableAtEnd() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
     if (gfshInstance == null) {
       fail("In testEcho command gfshInstance is null");
     }
-    LogWriterUtils.getLogWriter().info("Gsh " + gfshInstance);
+    getLogWriter().info("Gsh " + gfshInstance);
 
     gfshInstance.setEnvProperty("TESTSYS", "SYS_VALUE");
     printAllEnvs(gfshInstance);
@@ -122,6 +125,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testEchoWithNoVariable() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -146,6 +150,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testEchoWithVariableAtStart() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -169,6 +174,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testEchoWithMultipleVariables() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -192,6 +198,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testEchoAllPropertyVariables() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -210,6 +217,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testEchoForSingleVariable() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -234,6 +242,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testEchoForSingleVariable2() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -257,6 +266,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testDebug() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -278,6 +288,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
 
   }
 
+  @Test
   public void testHistory() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -297,6 +308,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testHistoryWithFileName() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -320,6 +332,7 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Test
   public void testClearHistory() {
     Gfsh gfshInstance = Gfsh.getCurrentInstance();
 
@@ -334,9 +347,9 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
 
     if (cmdResult != null) {
       assertEquals(Result.Status.OK, cmdResult.getStatus());
-      LogWriterUtils.getLogWriter().info("testClearHistory cmdResult=" + commandResultToString(cmdResult));
+      getLogWriter().info("testClearHistory cmdResult=" + commandResultToString(cmdResult));
       String resultString = commandResultToString(cmdResult);
-      LogWriterUtils.getLogWriter().info("testClearHistory resultString=" + resultString);
+      getLogWriter().info("testClearHistory resultString=" + resultString);
       assertTrue(resultString.contains(CliStrings.HISTORY__MSG__CLEARED_HISTORY));
       assertTrue(gfshInstance.getGfshHistory().size()<= 1);
     } else {
@@ -346,18 +359,18 @@ public class ShellCommandsDUnitTest extends CliCommandTestBase {
 
   private static void printCommandOutput(CommandResult cmdResult) {
     assertNotNull(cmdResult);
-    LogWriterUtils.getLogWriter().info("Command Output : ");
+    getLogWriter().info("Command Output : ");
     StringBuilder sb = new StringBuilder();
     cmdResult.resetToFirstLine();
     while (cmdResult.hasNextLine()) {
       sb.append(cmdResult.nextLine()).append(DataCommandRequest.NEW_LINE);
     }
-    LogWriterUtils.getLogWriter().info(sb.toString());
-    LogWriterUtils.getLogWriter().info("");
+    getLogWriter().info(sb.toString());
+    getLogWriter().info("");
   }
 
   private void printAllEnvs(Gfsh gfsh) {
-    LogWriterUtils.getLogWriter().info("printAllEnvs : " + StringUtils.objectToString(gfsh.getEnv(), false, 0));
+    getLogWriter().info("printAllEnvs : " + StringUtils.objectToString(gfsh.getEnv(), false, 0));
     /*
     getLogWriter().info("Gfsh printAllEnvs : " + HydraUtil.ObjectToString(getDefaultShell().getEnv()));    
     getLogWriter().info("gfsh " + gfsh + " default shell " + getDefaultShell());*/

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowDeadlockDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowDeadlockDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowDeadlockDUnitTest.java
index 6bf05b3..df01f14 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowDeadlockDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowDeadlockDUnitTest.java
@@ -16,12 +16,29 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.Invoke.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionContext;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
-import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.distributed.DistributedLockService;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.deadlock.GemFireDeadlockDetector;
@@ -32,34 +49,20 @@ import com.gemstone.gemfire.management.internal.cli.CliUtil;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.remote.CommandProcessor;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.Invoke;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
+import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * This DUnit tests uses same code as GemFireDeadlockDetectorDUnitTest and uses the command processor for executing the
  * "show deadlock" command
  */
-public class ShowDeadlockDUnitTest extends CacheTestCase {
+@Category(DistributedTest.class)
+public class ShowDeadlockDUnitTest extends JUnit4CacheTestCase {
 
-  /**
-   *
-   */
   private static final long serialVersionUID = 1L;
   private static final Set<Thread> stuckThreads = Collections.synchronizedSet(new HashSet<Thread>());
   private static final Map<String, String> EMPTY_ENV = Collections.emptyMap();
@@ -73,7 +76,7 @@ public class ShowDeadlockDUnitTest extends CacheTestCase {
 
   @Override
   public final void preTearDownCacheTestCase() throws Exception {
-    Invoke.invokeInEveryVM(new SerializableRunnable() {
+    invokeInEveryVM(new SerializableRunnable() {
       private static final long serialVersionUID = 1L;
 
       public void run() {
@@ -85,10 +88,7 @@ public class ShowDeadlockDUnitTest extends CacheTestCase {
     CliUtil.isGfshVM = true;
   }
 
-  public ShowDeadlockDUnitTest(String name) {
-    super(name);
-  }
-
+  @Test
   public void testNoDeadlock() throws ClassNotFoundException, IOException {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
@@ -112,7 +112,7 @@ public class ShowDeadlockDUnitTest extends CacheTestCase {
 
     String deadLockOutputFromCommand = getResultAsString(result);
 
-    LogWriterUtils.getLogWriter().info("output = " + deadLockOutputFromCommand);
+    getLogWriter().info("output = " + deadLockOutputFromCommand);
     assertEquals(true, result.hasIncomingFiles());
     assertEquals(true, result.getStatus().equals(Status.OK));
     assertEquals(true, deadLockOutputFromCommand.startsWith(CliStrings.SHOW_DEADLOCK__NO__DEADLOCK));
@@ -126,7 +126,7 @@ public class ShowDeadlockDUnitTest extends CacheTestCase {
 
   private static final Lock lock = new ReentrantLock();
 
-
+  @Test
   public void testDistributedDeadlockWithFunction() throws InterruptedException, ClassNotFoundException, IOException {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
@@ -148,7 +148,7 @@ public class ShowDeadlockDUnitTest extends CacheTestCase {
     Result result = commandProcessor.createCommandStatement(csb.toString(), EMPTY_ENV).process();
 
     String deadLockOutputFromCommand = getResultAsString(result);
-    LogWriterUtils.getLogWriter().info("Deadlock = " + deadLockOutputFromCommand);
+    getLogWriter().info("Deadlock = " + deadLockOutputFromCommand);
     result.saveIncomingFiles(null);
     assertEquals(true, deadLockOutputFromCommand.startsWith(CliStrings.SHOW_DEADLOCK__DEADLOCK__DETECTED));
     assertEquals(true, result.getStatus().equals(Status.OK));
@@ -185,7 +185,7 @@ public class ShowDeadlockDUnitTest extends CacheTestCase {
         try {
           Thread.sleep(1000);
         } catch (InterruptedException e) {
-          Assert.fail("interrupted", e);
+          fail("interrupted", e);
         }
         ResultCollector collector = FunctionService.onMember(basicGetSystem(), member).execute(new TestFunction());
         //wait the function to lock the lock on member.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java
index 90d0eed..c17f7e7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowMetricsDUnitTest.java
@@ -16,6 +16,19 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Properties;
+import javax.management.ObjectName;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionFactory;
@@ -36,30 +49,17 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.remote.CommandProcessor;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 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.DistributedTest;
 
-import javax.management.ObjectName;
-import java.io.File;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Properties;
-
-/****
- */
+@Category(DistributedTest.class)
 public class ShowMetricsDUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public ShowMetricsDUnitTest(String name) {
-    super(name);
-    // TODO Auto-generated constructor stub
-  }
-
   private void createLocalSetUp() {
     Properties localProps = new Properties();
     localProps.setProperty(DistributionConfig.NAME_NAME, "Controller");
@@ -70,9 +70,10 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
     Region region2 = dataRegionFactory.create("REGION2");
   }
 
-  /*
+  /**
    * tests the default version of "show metrics"
    */
+  @Test
   public void testShowMetricsDefault() {
     createDefaultSetup(null);
     createLocalSetUp();
@@ -96,11 +97,11 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
       @Override
       public Object call() throws Exception {
         WaitCriterion wc = createMBeanWaitCriterion(1, "", null, 0);
-        Wait.waitForCriterion(wc, 5000, 500, true);
+        waitForCriterion(wc, 5000, 500, true);
         CommandProcessor commandProcessor = new CommandProcessor();
         Result result = commandProcessor.createCommandStatement("show metrics", Collections.EMPTY_MAP).process();
         String resultStr = commandResultToString((CommandResult) result);
-        LogWriterUtils.getLogWriter().info(resultStr);
+        getLogWriter().info(resultStr);
         assertEquals(resultStr, true, result.getStatus().equals(Status.OK));
         return resultStr;
       }
@@ -112,8 +113,8 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
 
     String managerResult = (String) managerResultObj;
 
-    LogWriterUtils.getLogWriter().info("#SB Manager");
-    LogWriterUtils.getLogWriter().info(managerResult);
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
   }
 
   public void systemSetUp() {
@@ -135,6 +136,7 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
     });
   }
 
+  @Test
   public void testShowMetricsRegion() throws InterruptedException {
     systemSetUp();
     final String regionName = "REGION1";
@@ -143,7 +145,7 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
       @Override
       public Object call() throws Exception {
         WaitCriterion wc = createMBeanWaitCriterion(2, regionName, null, 0);
-        Wait.waitForCriterion(wc, 5000, 500, true);
+        waitForCriterion(wc, 5000, 500, true);
         CommandProcessor commandProcessor = new CommandProcessor();
         Result result = commandProcessor.createCommandStatement("show metrics --region=REGION1",
             Collections.EMPTY_MAP).process();
@@ -159,8 +161,8 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
 
     String managerResult = (String) managerResultObj;
 
-    LogWriterUtils.getLogWriter().info("#SB Manager");
-    LogWriterUtils.getLogWriter().info(managerResult);
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
   }
 
   /***
@@ -213,6 +215,7 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
     return waitCriterion;
   }
 
+  @Test
   public void testShowMetricsMember() throws ClassNotFoundException, IOException, InterruptedException {
     systemSetUp();
     Cache cache = getCache();
@@ -230,9 +233,9 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
       public Object call() throws Exception {
 
         WaitCriterion wc = createMBeanWaitCriterion(3, "", distributedMember, 0);
-        Wait.waitForCriterion(wc, 5000, 500, true);
+        waitForCriterion(wc, 5000, 500, true);
         wc = createMBeanWaitCriterion(5, "", distributedMember, cacheServerPort);
-        Wait.waitForCriterion(wc, 10000, 500, true);
+        waitForCriterion(wc, 10000, 500, true);
 
         final String command = CliStrings.SHOW_METRICS + " --" + CliStrings.SHOW_METRICS__MEMBER + "=" + distributedMember.getId() + " --" + CliStrings.SHOW_METRICS__CACHESERVER__PORT + "=" + cacheServerPort + " --" + CliStrings.SHOW_METRICS__FILE + "=" + exportFileName;
 
@@ -257,11 +260,12 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
 
     String managerResult = (String) managerResultObj;
 
-    LogWriterUtils.getLogWriter().info("#SB Manager");
-    LogWriterUtils.getLogWriter().info(managerResult);
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
     cs.stop();
   }
 
+  @Test
   public void testShowMetricsRegionFromMember() throws ClassNotFoundException, IOException, InterruptedException {
     systemSetUp();
     Cache cache = getCache();
@@ -275,7 +279,7 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
       public Object call() throws Exception {
 
         WaitCriterion wc = createMBeanWaitCriterion(4, regionName, distributedMember, 0);
-        Wait.waitForCriterion(wc, 5000, 500, true);
+        waitForCriterion(wc, 5000, 500, true);
         CommandProcessor commandProcessor = new CommandProcessor();
         Result result = commandProcessor.createCommandStatement(
             "show metrics --region=" + regionName + " --member=" + distributedMember.getName() + " --file=" + exportFileName,
@@ -298,10 +302,11 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
 
     String managerResult = (String) managerResultObj;
 
-    LogWriterUtils.getLogWriter().info("#SB Manager");
-    LogWriterUtils.getLogWriter().info(managerResult);
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
   }
 
+  @Test
   public void testShowMetricsRegionFromMemberWithCategories() throws ClassNotFoundException, IOException, InterruptedException {
     systemSetUp();
     Cache cache = getCache();
@@ -315,7 +320,7 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
       public Object call() throws Exception {
 
         WaitCriterion wc = createMBeanWaitCriterion(4, regionName, distributedMember, 0);
-        Wait.waitForCriterion(wc, 5000, 500, true);
+        waitForCriterion(wc, 5000, 500, true);
         CommandProcessor commandProcessor = new CommandProcessor();
         Result result = commandProcessor.createCommandStatement(
             "show metrics --region=" + regionName + " --member=" + distributedMember.getName() + " --file=" + exportFileName + " --categories=region,eviction",
@@ -338,7 +343,7 @@ public class ShowMetricsDUnitTest extends CliCommandTestBase {
 
     String managerResult = (String) managerResultObj;
 
-    LogWriterUtils.getLogWriter().info("#SB Manager");
-    LogWriterUtils.getLogWriter().info(managerResult);
+    getLogWriter().info("#SB Manager");
+    getLogWriter().info(managerResult);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java
index 9a72d36..6a14a40 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ShowStackTraceDUnitTest.java
@@ -16,32 +16,34 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.management.cli.Result.Status;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
 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 java.io.File;
-import java.io.IOException;
-import java.util.Properties;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /***
  * DUnit test for 'show stack-trace' command
- *
  */
+@Category(DistributedTest.class)
 public class ShowStackTraceDUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
 
-  public ShowStackTraceDUnitTest(String name) {
-    super(name);
-  }
-
   private void createCache(Properties props) {
     getSystem(props);
     getCache();
@@ -84,6 +86,7 @@ public class ShowStackTraceDUnitTest extends CliCommandTestBase {
    * @throws ClassNotFoundException
    * @throws IOException
    */
+  @Test
   public void testExportStacktrace() throws ClassNotFoundException, IOException {
     setupSystem();
 
@@ -93,9 +96,9 @@ public class ShowStackTraceDUnitTest extends CliCommandTestBase {
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.EXPORT_STACKTRACE);
     csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, allStacktracesFile.getCanonicalPath());
     String commandString = csb.toString();
-    LogWriterUtils.getLogWriter().info("CommandString : " + commandString);
+    getLogWriter().info("CommandString : " + commandString);
     CommandResult commandResult = executeCommand(commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
     assertTrue(commandResult.getStatus().equals(Status.OK));
 
     File mgrStacktraceFile = new File("managerStacktrace.txt");
@@ -105,9 +108,9 @@ public class ShowStackTraceDUnitTest extends CliCommandTestBase {
     csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, mgrStacktraceFile.getCanonicalPath());
     csb.addOption(CliStrings.EXPORT_STACKTRACE__MEMBER, "Manager");
     commandString = csb.toString();
-    LogWriterUtils.getLogWriter().info("CommandString : " + commandString);
+    getLogWriter().info("CommandString : " + commandString);
     commandResult = executeCommand(commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
     assertTrue(commandResult.getStatus().equals(Status.OK));
 
     File serverStacktraceFile = new File("serverStacktrace.txt");
@@ -117,9 +120,9 @@ public class ShowStackTraceDUnitTest extends CliCommandTestBase {
     csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, serverStacktraceFile.getCanonicalPath());
     csb.addOption(CliStrings.EXPORT_STACKTRACE__MEMBER, "Server");
     commandString = csb.toString();
-    LogWriterUtils.getLogWriter().info("CommandString : " + commandString);
+    getLogWriter().info("CommandString : " + commandString);
     commandResult = executeCommand(commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
     assertTrue(commandResult.getStatus().equals(Status.OK));
 
     File groupStacktraceFile = new File("groupstacktrace.txt");
@@ -129,9 +132,9 @@ public class ShowStackTraceDUnitTest extends CliCommandTestBase {
     csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, groupStacktraceFile.getCanonicalPath());
     csb.addOption(CliStrings.EXPORT_STACKTRACE__GROUP, "G2");
     commandString = csb.toString();
-    LogWriterUtils.getLogWriter().info("CommandString : " + commandString);
+    getLogWriter().info("CommandString : " + commandString);
     commandResult = executeCommand(commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
     assertTrue(commandResult.getStatus().equals(Status.OK));
 
     File wrongStackTraceFile = new File("wrongStackTrace.txt");
@@ -141,9 +144,9 @@ public class ShowStackTraceDUnitTest extends CliCommandTestBase {
     csb.addOption(CliStrings.EXPORT_STACKTRACE__FILE, wrongStackTraceFile.getCanonicalPath());
     csb.addOption(CliStrings.EXPORT_STACKTRACE__MEMBER, "WrongMember");
     commandString = csb.toString();
-    LogWriterUtils.getLogWriter().info("CommandString : " + commandString);
+    getLogWriter().info("CommandString : " + commandString);
     commandResult = executeCommand(commandString);
-    LogWriterUtils.getLogWriter().info("Output : \n" + commandResultToString(commandResult));
+    getLogWriter().info("Output : \n" + commandResultToString(commandResult));
     assertFalse(commandResult.getStatus().equals(Status.OK));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java
index 97a9305..3b4db25 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/UserCommandsDUnitTest.java
@@ -16,6 +16,16 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Properties;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.ClassBuilder;
 import com.gemstone.gemfire.internal.ClassPathLoader;
@@ -25,20 +35,16 @@ import com.gemstone.gemfire.management.internal.cli.CommandManager;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
-
-import org.junit.Test;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.Properties;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Unit tests for configuring user commands.
  *
  * @since 8.0
  */
+@Category(DistributedTest.class)
 public class UserCommandsDUnitTest extends CliCommandTestBase {
+
   private static final long serialVersionUID = 1L;
   final File jarDirectory = new File(
       (new File(ClassPathLoader.class.getProtectionDomain().getCodeSource().getLocation().getPath())).getParent(),
@@ -46,10 +52,6 @@ public class UserCommandsDUnitTest extends CliCommandTestBase {
   final File jarFile = new File(this.jarDirectory, "UserCommandsDUnit.jar");
   boolean deleteJarDirectory = false;
 
-  public UserCommandsDUnitTest(String name) throws Exception {
-    super(name);
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     createUserCommandJarFile();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
index ea83a66..83bf532 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationDUnitTest.java
@@ -18,11 +18,13 @@
  */
 package com.gemstone.gemfire.security;
 
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
+
 /**
  * Test for authentication from client to server. This tests for both valid and
  * invalid credentials/modules. It also checks for authentication
@@ -30,7 +32,7 @@ import org.junit.experimental.categories.Category;
  * 
  * @since 5.5
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class ClientAuthenticationDUnitTest extends ClientAuthenticationTestCase {
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
index b633865..24fcc3f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationPart2DUnitTest.java
@@ -18,16 +18,18 @@
  */
 package com.gemstone.gemfire.security;
 
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
+
 /**
  * this class contains test methods that used to be in its superclass but
  * that test started taking too long and caused dunit runs to hang
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class ClientAuthenticationPart2DUnitTest extends ClientAuthenticationTestCase {
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestCase.java
index 191ea3e..381bc33 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthenticationTestCase.java
@@ -17,8 +17,11 @@
 package com.gemstone.gemfire.security;
 
 import static com.gemstone.gemfire.internal.AvailablePort.*;
+import static com.gemstone.gemfire.security.ClientAuthenticationTestUtils.createCacheClient;
+import static com.gemstone.gemfire.security.ClientAuthenticationTestUtils.createCacheServer;
 import static com.gemstone.gemfire.security.ClientAuthenticationTestUtils.*;
 import static com.gemstone.gemfire.security.SecurityTestUtils.*;
+import static com.gemstone.gemfire.security.SecurityTestUtils.createCacheClient;
 import static com.gemstone.gemfire.test.dunit.IgnoredException.*;
 import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
 import static com.gemstone.gemfire.test.dunit.Wait.*;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
index eeb2c39..961e844 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationDUnitTest.java
@@ -19,8 +19,6 @@
 package com.gemstone.gemfire.security;
 
 import static com.gemstone.gemfire.internal.AvailablePort.*;
-//import static com.gemstone.gemfire.security.ClientAuthenticationTestUtils.*;
-//import static com.gemstone.gemfire.security.ClientAuthorizationTestCase.*;
 import static com.gemstone.gemfire.security.SecurityTestUtils.*;
 import static com.gemstone.gemfire.test.dunit.Assert.*;
 import static com.gemstone.gemfire.test.dunit.IgnoredException.*;
@@ -31,6 +29,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 import com.gemstone.gemfire.security.generator.AuthzCredentialGenerator;
 import com.gemstone.gemfire.security.generator.CredentialGenerator;
@@ -39,8 +40,7 @@ import com.gemstone.gemfire.security.generator.XmlAuthzCredentialGenerator;
 import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 
 /**
  * Tests for authorization from client to server. This tests for authorization
@@ -50,7 +50,7 @@ import org.junit.experimental.categories.Category;
  * 
  * @since 5.5
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class ClientAuthorizationDUnitTest extends ClientAuthorizationTestCase {
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
index 088dec4..4cc8155 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTestCase.java
@@ -55,7 +55,7 @@ import com.gemstone.gemfire.cache.query.QueryInvocationTargetException;
 import com.gemstone.gemfire.cache.query.QueryService;
 import com.gemstone.gemfire.cache.query.SelectResults;
 import com.gemstone.gemfire.cache.query.Struct;
-import com.gemstone.gemfire.internal.AvailablePort.Keeper;
+import com.gemstone.gemfire.internal.AvailablePort.*;
 import com.gemstone.gemfire.internal.cache.AbstractRegionEntry;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.security.generator.AuthzCredentialGenerator;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
index 441d52c..38f9988 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/ClientMultiUserAuthzDUnitTest.java
@@ -18,14 +18,15 @@
  */
 package com.gemstone.gemfire.security;
 
-//import static com.gemstone.gemfire.security.ClientAuthenticationTestUtils.*;
-//import static com.gemstone.gemfire.security.ClientAuthorizationTestCase.*;
 import static com.gemstone.gemfire.security.SecurityTestUtils.*;
 import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
 
 import java.util.Iterator;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
@@ -36,10 +37,9 @@ import com.gemstone.gemfire.security.generator.AuthzCredentialGenerator;
 import com.gemstone.gemfire.security.generator.CredentialGenerator;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class ClientMultiUserAuthzDUnitTest extends ClientAuthorizationTestCase {
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
index 904a53c..3e399b7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientAuthorizationDUnitTest.java
@@ -18,13 +18,16 @@
  */
 package com.gemstone.gemfire.security;
 
-import static com.gemstone.gemfire.security.ClientAuthenticationTestUtils.*;
+import static com.gemstone.gemfire.security.ClientAuthenticationTestUtils.createCacheClient;
 import static com.gemstone.gemfire.security.SecurityTestUtils.*;
 import static com.gemstone.gemfire.test.dunit.Assert.*;
 import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
 
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.DeltaTestImpl;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
@@ -32,13 +35,12 @@ import com.gemstone.gemfire.internal.cache.PartitionedRegionLocalMaxMemoryDUnitT
 import com.gemstone.gemfire.security.generator.AuthzCredentialGenerator;
 import com.gemstone.gemfire.security.generator.CredentialGenerator;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 
 /**
  * @since 6.1
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public final class DeltaClientAuthorizationDUnitTest extends ClientAuthorizationTestCase {
 
   private DeltaTestImpl[] deltas = new DeltaTestImpl[8];

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
index 222ea00..61ff55a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/DeltaClientPostAuthorizationDUnitTest.java
@@ -30,6 +30,9 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.DeltaTestImpl;
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionLocalMaxMemoryDUnitTest;
@@ -37,13 +40,12 @@ import com.gemstone.gemfire.security.generator.AuthzCredentialGenerator;
 import com.gemstone.gemfire.security.generator.CredentialGenerator;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 
 /**
  * @since 6.1
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class DeltaClientPostAuthorizationDUnitTest extends ClientAuthorizationTestCase {
 
   private static final int PAUSE = 5 * 1000; // TODO: replace with Awaitility



[04/32] incubator-geode git commit: GEODE-1221 Move no longer supported use case to test tree.

Posted by ji...@apache.org.
GEODE-1221 Move no longer supported use case to test tree.


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 6c033cfe75de2e42d97bacf7cae122fc1adc8600
Parents: 90ab09c
Author: eshu <es...@pivotal.io>
Authored: Wed Apr 13 10:02:58 2016 -0700
Committer: eshu <es...@pivotal.io>
Committed: Wed Apr 13 10:02:58 2016 -0700

----------------------------------------------------------------------
 .../cache/execute/util/CommitFunction.java      | 141 ------------------
 .../execute/util/NestedTransactionFunction.java | 116 ---------------
 .../cache/execute/util/RollbackFunction.java    | 136 ------------------
 .../cache/ClientServerTransactionDUnitTest.java |   2 -
 .../gemfire/internal/cache/CommitFunction.java  | 142 +++++++++++++++++++
 .../cache/NestedTransactionFunction.java        | 116 +++++++++++++++
 .../internal/cache/RollbackFunction.java        | 137 ++++++++++++++++++
 7 files changed, 395 insertions(+), 395 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c033cfe/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/CommitFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/CommitFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/CommitFunction.java
deleted file mode 100644
index 23139b4..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/CommitFunction.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.cache.execute.util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.CacheTransactionManager;
-import com.gemstone.gemfire.cache.TransactionDataNodeHasDepartedException;
-import com.gemstone.gemfire.cache.TransactionId;
-import com.gemstone.gemfire.cache.execute.Execution;
-import com.gemstone.gemfire.cache.execute.Function;
-import com.gemstone.gemfire.cache.execute.FunctionContext;
-import com.gemstone.gemfire.cache.execute.FunctionException;
-import com.gemstone.gemfire.cache.execute.FunctionInvocationTargetException;
-import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.internal.cache.TXId;
-import com.gemstone.gemfire.internal.logging.LogService;
-
-/**
- * This function can be used by GemFire clients and peers to commit an existing
- * transaction. A {@link TransactionId} corresponding to the transaction to be
- * committed must be provided as an argument while invoking this function.<br />
- * 
- * This function should execute only on one server. If the transaction is not
- * hosted on the server where the function is invoked then this function decides
- * to invoke a nested {@link NestedTransactionFunction} which executes on the member where
- * transaction is hosted.<br />
- * 
- * This function returns a single Boolean as result, whose value is <code>Boolean.TRUE</code>
- * if the transaction committed successfully otherwise the return value is
- * <code>Boolean.FALSE</code>.<br />
- * 
- * To execute this function, it is recommended to use the {@link Execution} obtained by
- * using TransactionFunctionService. <br />
- * 
- * To summarize, this function should be used as follows:
- * 
- * <pre>
- * Execution exe = TransactionFunctionService.onTransaction(txId);
- * List l = (List) exe.execute(commitFunction).getResult();
- * Boolean result = (Boolean) l.get(0);
- * </pre>
- * 
- * This function is <b>not</b> registered on the cache servers by default, and
- * it is the user's responsibility to register this function. see
- * {@link FunctionService#registerFunction(Function)}
- * 
- * @since 6.6.1
- */
-public class CommitFunction implements Function {
-  private static final Logger logger = LogService.getLogger();
-
-  private static final long serialVersionUID = 7851518767859544501L;
-
-  public boolean hasResult() {
-    return true;
-  }
-
-  public void execute(FunctionContext context) {
-    Cache cache = CacheFactory.getAnyInstance();
-    TXId txId = null;
-    try {
-      txId = (TXId) context.getArguments();
-    } catch (ClassCastException e) {
-      logger.info("CommitFunction should be invoked with a TransactionId as an argument i.e. withArgs(txId).execute(function)");
-      throw e;
-    }
-    DistributedMember member = txId.getMemberId();
-    Boolean result = false;
-    final boolean isDebugEnabled = logger.isDebugEnabled();
-    if (cache.getDistributedSystem().getDistributedMember().equals(member)) {
-      if (isDebugEnabled) {
-        logger.debug("CommitFunction: for transaction: {} committing locally", txId);
-      }
-      CacheTransactionManager txMgr = cache.getCacheTransactionManager();
-      if (txMgr.tryResume(txId)) {
-        if (isDebugEnabled) {
-          logger.debug("CommitFunction: resumed transaction: {}", txId);
-        }
-        txMgr.commit();
-        result = true;
-      }
-    } else {
-      ArrayList args = new ArrayList();
-      args.add(txId);
-      args.add(NestedTransactionFunction.COMMIT);
-      Execution ex = FunctionService.onMember(cache.getDistributedSystem(),
-          member).withArgs(args);
-      if (isDebugEnabled) {
-        logger.debug("CommitFunction: for transaction: {} executing NestedTransactionFunction on member: {}", txId, member);
-      }
-      try {
-        List list = (List) ex.execute(new NestedTransactionFunction()).getResult();
-        result = (Boolean) list.get(0);
-      } catch (FunctionException fe) {
-        if (fe.getCause() instanceof FunctionInvocationTargetException) {
-          throw new TransactionDataNodeHasDepartedException("Could not commit on member:"+member);
-        } else {
-          throw fe;
-        }
-      }
-    }
-    if (isDebugEnabled) {
-      logger.debug("CommitFunction: for transaction: {} returning result: {}", txId, result);
-    }
-    context.getResultSender().lastResult(result);
-  }
-
-  public String getId() {
-    return getClass().getName();
-  }
-
-  public boolean optimizeForWrite() {
-    return true;
-  }
-
-  public boolean isHA() {
-    return false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c033cfe/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/NestedTransactionFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/NestedTransactionFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/NestedTransactionFunction.java
deleted file mode 100644
index 9e4aa9d..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/NestedTransactionFunction.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.cache.execute.util;
-
-import java.util.ArrayList;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.CacheTransactionManager;
-import com.gemstone.gemfire.cache.TransactionId;
-import com.gemstone.gemfire.cache.execute.Function;
-import com.gemstone.gemfire.cache.execute.FunctionContext;
-import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.internal.cache.TXId;
-import com.gemstone.gemfire.internal.logging.LogService;
-
-/**
- * This function is used by {@link CommitFunction} to commit existing transaction.
- * A {@link TransactionId} corresponding to the transaction to be
- * committed must be provided as an argument while invoking this function.<br />
- * 
- * When executed this function commits a transaction if it exists locally.<br />
- * 
- * This function returns a single Boolean as result, whose value is <code>Boolean.TRUE</code>
- * if the transaction committed successfully otherwise the return value is
- * <code>Boolean.FALSE</code><br />
- * 
- * This function is <b>not</b> registered on the cache servers by default, and
- * it is the user's responsibility to register this function. see
- * {@link FunctionService#registerFunction(Function)}
- * 
- * @see CommitFunction
- * @since 6.6.1
- *
- */
-public class NestedTransactionFunction implements Function {
-  private static final Logger logger = LogService.getLogger();
-
-  public static final int COMMIT = 1;
-  public static final int ROLLBACK = 2;
-  
-  private static final long serialVersionUID = 1400965724856341543L;
-
-  public boolean hasResult() {
-    return true;
-  }
-
-  public void execute(FunctionContext context) {
-    Cache cache = CacheFactory.getAnyInstance();
-    ArrayList args = (ArrayList) context.getArguments();
-    TXId txId = null;
-    int action = 0;
-    try {
-      txId = (TXId) args.get(0);
-      action = (Integer) args.get(1);
-    } catch (ClassCastException e) {
-      logger.info("CommitFunction should be invoked with a TransactionId as an argument i.e. withArgs(txId).execute(function)");
-      throw e;
-    }
-    CacheTransactionManager txMgr = cache.getCacheTransactionManager();
-    Boolean result = false;
-    final boolean isDebugEnabled = logger.isDebugEnabled();
-    if (txMgr.tryResume(txId)) {
-      if (isDebugEnabled) {
-        logger.debug("CommitFunction: resumed transaction: {}", txId);
-      }
-      if (action == COMMIT) {
-        if (isDebugEnabled) {
-          logger.debug("CommitFunction: committing transaction: {}", txId);
-        }
-        txMgr.commit();
-      } else if (action == ROLLBACK) {
-        if (isDebugEnabled) {
-          logger.debug("CommitFunction: rolling back transaction: {}", txId);
-        }
-        txMgr.rollback();
-      } else {
-        throw new IllegalStateException("unknown transaction termination action");
-      }
-      result = true;
-    }
-    if (isDebugEnabled) {
-      logger.debug("CommitFunction: for transaction: {} sending result: {}", txId, result);
-    }
-    context.getResultSender().lastResult(result);
-  }
-
-  public String getId() {
-    return getClass().getName();
-  }
-
-  public boolean optimizeForWrite() {
-    return true;
-  }
-
-  public boolean isHA() {
-    return false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c033cfe/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/RollbackFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/RollbackFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/RollbackFunction.java
deleted file mode 100644
index f413d6c..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/RollbackFunction.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.cache.execute.util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.CacheTransactionManager;
-import com.gemstone.gemfire.cache.TransactionDataNodeHasDepartedException;
-import com.gemstone.gemfire.cache.TransactionId;
-import com.gemstone.gemfire.cache.execute.Execution;
-import com.gemstone.gemfire.cache.execute.Function;
-import com.gemstone.gemfire.cache.execute.FunctionContext;
-import com.gemstone.gemfire.cache.execute.FunctionException;
-import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.internal.cache.TXId;
-import com.gemstone.gemfire.internal.logging.LogService;
-
-/**
- * This function can be used by GemFire clients and peers to rollback an existing
- * transaction. A {@link TransactionId} corresponding to the transaction to be
- * rolledback must be provided as an argument while invoking this function.<br />
- * 
- * This function should execute only on one server. If the transaction is not
- * hosted on the server where the function is invoked then this function decides
- * to invoke a {@link NestedTransactionFunction} which executes on the member where
- * transaction is hosted.<br />
- * 
- * This function returns a single Boolean as result, whose value is <code>Boolean.TRUE</code>
- * if the transaction rolled back successfully otherwise the return value is
- * <code>Boolean.FALSE</code>.<br />
- * 
- * To execute this function, it is recommended to use the {@link Execution} obtained by
- * using TransactionFunctionService. <br />
- * 
- * To summarize, this function should be used as follows:
- * 
- * <pre>
- * Execution exe = TransactionFunctionService.onTransaction(txId);
- * List l = (List) exe.execute(rollbackFunction).getResult();
- * Boolean result = (Boolean) l.get(0);
- * </pre>
- * 
- * This function is <b>not</b> registered on the cache servers by default, and
- * it is the user's responsibility to register this function. see
- * {@link FunctionService#registerFunction(Function)}
- * 
- * @since 6.6.1
- */
-public class RollbackFunction implements Function {
-  private static final Logger logger = LogService.getLogger();
-
-  private static final long serialVersionUID = 1377183180063184795L;
-
-  public boolean hasResult() {
-    return true;
-  }
-
-  public void execute(FunctionContext context) {
-    Cache cache = CacheFactory.getAnyInstance();
-    TXId txId = null;
-    try {
-      txId = (TXId) context.getArguments();
-    } catch (ClassCastException e) {
-      logger.info("RollbackFunction should be invoked with a TransactionId as an argument i.e. withArgs(txId).execute(function)");
-      throw e;
-    }
-    DistributedMember member = txId.getMemberId();
-    Boolean result = false;
-    final boolean isDebugEnabled = logger.isDebugEnabled();
-    if (cache.getDistributedSystem().getDistributedMember().equals(member)) {
-      if (isDebugEnabled) {
-        logger.debug("RollbackFunction: for transaction: {} rolling back locally", txId);
-      }
-      CacheTransactionManager txMgr = cache.getCacheTransactionManager();
-      if (txMgr.tryResume(txId)) {
-        if (isDebugEnabled) {
-          logger.debug("RollbackFunction: resumed transaction: {}", txId);
-        }
-        txMgr.rollback();
-        result = true;
-      }
-    } else {
-      ArrayList args = new ArrayList();
-      args.add(txId);
-      args.add(NestedTransactionFunction.ROLLBACK);
-      Execution ex = FunctionService.onMember(cache.getDistributedSystem(),
-          member).withArgs(args);
-      if (isDebugEnabled) {
-        logger.debug("RollbackFunction: for transaction: {} executing NestedTransactionFunction on member: {}", txId, member);
-      }
-      try {
-        List list = (List) ex.execute(new NestedTransactionFunction()).getResult();
-        result = (Boolean) list.get(0);
-      } catch (FunctionException fe) {
-        throw new TransactionDataNodeHasDepartedException("Could not Rollback on member:"+member);
-      }
-    }
-    if (isDebugEnabled) {
-      logger.debug("RollbackFunction: for transaction: {} returning result: {}", txId, result);
-    }
-    context.getResultSender().lastResult(result);
-  }
-
-  public String getId() {
-    return getClass().getName();
-  }
-
-  public boolean optimizeForWrite() {
-    return true;
-  }
-
-  public boolean isHA() {
-    return false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c033cfe/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
index 926a50a..672c5e6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
@@ -85,8 +85,6 @@ import com.gemstone.gemfire.internal.cache.execute.data.CustId;
 import com.gemstone.gemfire.internal.cache.execute.data.Customer;
 import com.gemstone.gemfire.internal.cache.execute.data.Order;
 import com.gemstone.gemfire.internal.cache.execute.data.OrderId;
-import com.gemstone.gemfire.internal.cache.execute.util.CommitFunction;
-import com.gemstone.gemfire.internal.cache.execute.util.RollbackFunction;
 import com.gemstone.gemfire.internal.cache.tx.ClientTXStateStub;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.IgnoredException;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c033cfe/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/CommitFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/CommitFunction.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/CommitFunction.java
new file mode 100644
index 0000000..051465d
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/CommitFunction.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.CacheTransactionManager;
+import com.gemstone.gemfire.cache.TransactionDataNodeHasDepartedException;
+import com.gemstone.gemfire.cache.TransactionId;
+import com.gemstone.gemfire.cache.execute.Execution;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.execute.FunctionException;
+import com.gemstone.gemfire.cache.execute.FunctionInvocationTargetException;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.internal.cache.TXId;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+/**
+ * This function can be used by GemFire clients and peers to commit an existing
+ * transaction. A {@link TransactionId} corresponding to the transaction to be
+ * committed must be provided as an argument while invoking this function.<br />
+ * 
+ * This function should execute only on one server. If the transaction is not
+ * hosted on the server where the function is invoked then this function decides
+ * to invoke a nested {@link NestedTransactionFunction} which executes on the member where
+ * transaction is hosted.<br />
+ * 
+ * This function returns a single Boolean as result, whose value is <code>Boolean.TRUE</code>
+ * if the transaction committed successfully otherwise the return value is
+ * <code>Boolean.FALSE</code>.<br />
+ * 
+ * To execute this function, it is recommended to use the {@link Execution} obtained by
+ * using TransactionFunctionService. <br />
+ * 
+ * To summarize, this function should be used as follows:
+ * 
+ * <pre>
+ * Execution exe = TransactionFunctionService.onTransaction(txId);
+ * List l = (List) exe.execute(commitFunction).getResult();
+ * Boolean result = (Boolean) l.get(0);
+ * </pre>
+ * 
+ * This function is <b>not</b> registered on the cache servers by default, and
+ * it is the user's responsibility to register this function. see
+ * {@link FunctionService#registerFunction(Function)}
+ * 
+ * @since 6.6.1
+ */
+public class CommitFunction implements Function {
+  private static final Logger logger = LogService.getLogger();
+
+  private static final long serialVersionUID = 7851518767859544501L;
+
+  public boolean hasResult() {
+    return true;
+  }
+
+  public void execute(FunctionContext context) {
+    Cache cache = CacheFactory.getAnyInstance();
+    TXId txId = null;
+    try {
+      txId = (TXId) context.getArguments();
+    } catch (ClassCastException e) {
+      logger.info("CommitFunction should be invoked with a TransactionId as an argument i.e. withArgs(txId).execute(function)");
+      throw e;
+    }
+    DistributedMember member = txId.getMemberId();
+    Boolean result = false;
+    final boolean isDebugEnabled = logger.isDebugEnabled();
+    if (cache.getDistributedSystem().getDistributedMember().equals(member)) {
+      if (isDebugEnabled) {
+        logger.debug("CommitFunction: for transaction: {} committing locally", txId);
+      }
+      CacheTransactionManager txMgr = cache.getCacheTransactionManager();
+      if (txMgr.tryResume(txId)) {
+        if (isDebugEnabled) {
+          logger.debug("CommitFunction: resumed transaction: {}", txId);
+        }
+        txMgr.commit();
+        result = true;
+      }
+    } else {
+      ArrayList args = new ArrayList();
+      args.add(txId);
+      args.add(NestedTransactionFunction.COMMIT);
+      Execution ex = FunctionService.onMember(cache.getDistributedSystem(),
+          member).withArgs(args);
+      if (isDebugEnabled) {
+        logger.debug("CommitFunction: for transaction: {} executing NestedTransactionFunction on member: {}", txId, member);
+      }
+      try {
+        List list = (List) ex.execute(new NestedTransactionFunction()).getResult();
+        result = (Boolean) list.get(0);
+      } catch (FunctionException fe) {
+        if (fe.getCause() instanceof FunctionInvocationTargetException) {
+          throw new TransactionDataNodeHasDepartedException("Could not commit on member:"+member);
+        } else {
+          throw fe;
+        }
+      }
+    }
+    if (isDebugEnabled) {
+      logger.debug("CommitFunction: for transaction: {} returning result: {}", txId, result);
+    }
+    context.getResultSender().lastResult(result);
+  }
+
+  public String getId() {
+    return getClass().getName();
+  }
+
+  public boolean optimizeForWrite() {
+    return true;
+  }
+
+  public boolean isHA() {
+    //GEM-207
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c033cfe/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/NestedTransactionFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/NestedTransactionFunction.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/NestedTransactionFunction.java
new file mode 100644
index 0000000..edf259c
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/NestedTransactionFunction.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache;
+
+import java.util.ArrayList;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.CacheTransactionManager;
+import com.gemstone.gemfire.cache.TransactionId;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.internal.cache.TXId;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+/**
+ * This function is used by {@link CommitFunction} to commit existing transaction.
+ * A {@link TransactionId} corresponding to the transaction to be
+ * committed must be provided as an argument while invoking this function.<br />
+ * 
+ * When executed this function commits a transaction if it exists locally.<br />
+ * 
+ * This function returns a single Boolean as result, whose value is <code>Boolean.TRUE</code>
+ * if the transaction committed successfully otherwise the return value is
+ * <code>Boolean.FALSE</code><br />
+ * 
+ * This function is <b>not</b> registered on the cache servers by default, and
+ * it is the user's responsibility to register this function. see
+ * {@link FunctionService#registerFunction(Function)}
+ * 
+ * @see CommitFunction
+ * @since 6.6.1
+ *
+ */
+public class NestedTransactionFunction implements Function {
+  private static final Logger logger = LogService.getLogger();
+
+  public static final int COMMIT = 1;
+  public static final int ROLLBACK = 2;
+  
+  private static final long serialVersionUID = 1400965724856341543L;
+
+  public boolean hasResult() {
+    return true;
+  }
+
+  public void execute(FunctionContext context) {
+    Cache cache = CacheFactory.getAnyInstance();
+    ArrayList args = (ArrayList) context.getArguments();
+    TXId txId = null;
+    int action = 0;
+    try {
+      txId = (TXId) args.get(0);
+      action = (Integer) args.get(1);
+    } catch (ClassCastException e) {
+      logger.info("CommitFunction should be invoked with a TransactionId as an argument i.e. withArgs(txId).execute(function)");
+      throw e;
+    }
+    CacheTransactionManager txMgr = cache.getCacheTransactionManager();
+    Boolean result = false;
+    final boolean isDebugEnabled = logger.isDebugEnabled();
+    if (txMgr.tryResume(txId)) {
+      if (isDebugEnabled) {
+        logger.debug("CommitFunction: resumed transaction: {}", txId);
+      }
+      if (action == COMMIT) {
+        if (isDebugEnabled) {
+          logger.debug("CommitFunction: committing transaction: {}", txId);
+        }
+        txMgr.commit();
+      } else if (action == ROLLBACK) {
+        if (isDebugEnabled) {
+          logger.debug("CommitFunction: rolling back transaction: {}", txId);
+        }
+        txMgr.rollback();
+      } else {
+        throw new IllegalStateException("unknown transaction termination action");
+      }
+      result = true;
+    }
+    if (isDebugEnabled) {
+      logger.debug("CommitFunction: for transaction: {} sending result: {}", txId, result);
+    }
+    context.getResultSender().lastResult(result);
+  }
+
+  public String getId() {
+    return getClass().getName();
+  }
+
+  public boolean optimizeForWrite() {
+    return true;
+  }
+
+  public boolean isHA() {
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c033cfe/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/RollbackFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/RollbackFunction.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/RollbackFunction.java
new file mode 100644
index 0000000..7a319a2
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/RollbackFunction.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.CacheTransactionManager;
+import com.gemstone.gemfire.cache.TransactionDataNodeHasDepartedException;
+import com.gemstone.gemfire.cache.TransactionId;
+import com.gemstone.gemfire.cache.execute.Execution;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.execute.FunctionException;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.internal.cache.TXId;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+/**
+ * This function can be used by GemFire clients and peers to rollback an existing
+ * transaction. A {@link TransactionId} corresponding to the transaction to be
+ * rolledback must be provided as an argument while invoking this function.<br />
+ * 
+ * This function should execute only on one server. If the transaction is not
+ * hosted on the server where the function is invoked then this function decides
+ * to invoke a {@link NestedTransactionFunction} which executes on the member where
+ * transaction is hosted.<br />
+ * 
+ * This function returns a single Boolean as result, whose value is <code>Boolean.TRUE</code>
+ * if the transaction rolled back successfully otherwise the return value is
+ * <code>Boolean.FALSE</code>.<br />
+ * 
+ * To execute this function, it is recommended to use the {@link Execution} obtained by
+ * using TransactionFunctionService. <br />
+ * 
+ * To summarize, this function should be used as follows:
+ * 
+ * <pre>
+ * Execution exe = TransactionFunctionService.onTransaction(txId);
+ * List l = (List) exe.execute(rollbackFunction).getResult();
+ * Boolean result = (Boolean) l.get(0);
+ * </pre>
+ * 
+ * This function is <b>not</b> registered on the cache servers by default, and
+ * it is the user's responsibility to register this function. see
+ * {@link FunctionService#registerFunction(Function)}
+ * 
+ * @since 6.6.1
+ */
+public class RollbackFunction implements Function {
+  private static final Logger logger = LogService.getLogger();
+
+  private static final long serialVersionUID = 1377183180063184795L;
+
+  public boolean hasResult() {
+    return true;
+  }
+
+  public void execute(FunctionContext context) {
+    Cache cache = CacheFactory.getAnyInstance();
+    TXId txId = null;
+    try {
+      txId = (TXId) context.getArguments();
+    } catch (ClassCastException e) {
+      logger.info("RollbackFunction should be invoked with a TransactionId as an argument i.e. withArgs(txId).execute(function)");
+      throw e;
+    }
+    DistributedMember member = txId.getMemberId();
+    Boolean result = false;
+    final boolean isDebugEnabled = logger.isDebugEnabled();
+    if (cache.getDistributedSystem().getDistributedMember().equals(member)) {
+      if (isDebugEnabled) {
+        logger.debug("RollbackFunction: for transaction: {} rolling back locally", txId);
+      }
+      CacheTransactionManager txMgr = cache.getCacheTransactionManager();
+      if (txMgr.tryResume(txId)) {
+        if (isDebugEnabled) {
+          logger.debug("RollbackFunction: resumed transaction: {}", txId);
+        }
+        txMgr.rollback();
+        result = true;
+      }
+    } else {
+      ArrayList args = new ArrayList();
+      args.add(txId);
+      args.add(NestedTransactionFunction.ROLLBACK);
+      Execution ex = FunctionService.onMember(cache.getDistributedSystem(),
+          member).withArgs(args);
+      if (isDebugEnabled) {
+        logger.debug("RollbackFunction: for transaction: {} executing NestedTransactionFunction on member: {}", txId, member);
+      }
+      try {
+        List list = (List) ex.execute(new NestedTransactionFunction()).getResult();
+        result = (Boolean) list.get(0);
+      } catch (FunctionException fe) {
+        throw new TransactionDataNodeHasDepartedException("Could not Rollback on member:"+member);
+      }
+    }
+    if (isDebugEnabled) {
+      logger.debug("RollbackFunction: for transaction: {} returning result: {}", txId, result);
+    }
+    context.getResultSender().lastResult(result);
+  }
+
+  public String getId() {
+    return getClass().getName();
+  }
+
+  public boolean optimizeForWrite() {
+    return true;
+  }
+
+  public boolean isHA() {
+    //GEM-207
+    return true;
+  }
+
+}


[19/32] incubator-geode git commit: GEODE-1162: convert all CLI command DUnit tests to JUnit 4

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
index f1afdaf..6fef000 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/ClientCommandsDUnitTest.java
@@ -16,16 +16,23 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.DistributedTestUtils.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.NetworkUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
 
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Map.Entry;
-
+import java.util.Properties;
 import javax.management.ObjectName;
 
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheException;
@@ -63,23 +70,20 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData;
 import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData.SectionResultData;
-import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
+import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
-import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
-
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
 /**
  * Dunit class for testing gemfire Client commands : list client , describe client 
  * @since 8.0
  */
-
-
+@Category({ DistributedTest.class, FlakyTest.class }) // see GEODE-1034
 public class ClientCommandsDUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
@@ -92,15 +96,7 @@ public class ClientCommandsDUnitTest extends CliCommandTestBase {
   int port0 = 0;
   int port1= 0;
   
-  
-  
-  public ClientCommandsDUnitTest(String name) {
-    super(name);
-    
-  }
-
-  
-public void waitForListClientMbean(){
+  public void waitForListClientMbean(){
     
     final VM manager = Host.getHost(0).getVM(0);
     final VM server1 = Host.getHost(0).getVM(1);
@@ -117,7 +113,7 @@ public void waitForListClientMbean(){
           public boolean done() {
             final SystemManagementService service = (SystemManagementService) ManagementService.getManagementService(getCache());
             if (service == null) {
-              Log.getLogWriter().info("waitForListClientMbean Still probing for service");
+              getLogWriter().info("waitForListClientMbean Still probing for service");
               return false;
             } else {      
               final ObjectName cacheServerMBeanName = service.getCacheServerMBeanName(port0,serverMember);                            
@@ -142,7 +138,7 @@ public void waitForListClientMbean(){
             return "waitForListClientMbean Probing ...";
           }
         };
-        Wait.waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+        waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
       }
     }); 
     
@@ -166,7 +162,7 @@ public void waitForListClientMbean2(){
         public boolean done() {
           final SystemManagementService service = (SystemManagementService) ManagementService.getManagementService(getCache());
           if (service == null) {
-            Log.getLogWriter().info("waitForListClientMbean2 Still probing for service");
+            getLogWriter().info("waitForListClientMbean2 Still probing for service");
             return false;
           } else {      
             final ObjectName cacheServerMBeanName = service.getCacheServerMBeanName(port0,serverMember);                            
@@ -191,7 +187,7 @@ public void waitForListClientMbean2(){
           return "waitForListClientMbean2 Probing ...";
         }
       };
-      Wait.waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+      waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
     }
   }); 
   
@@ -216,7 +212,7 @@ public void waitForListClientMbean2(){
           public boolean done() {
             final SystemManagementService service = (SystemManagementService) ManagementService.getManagementService(getCache());
             if (service == null) {
-              Log.getLogWriter().info("waitForMbean Still probing for service");
+              getLogWriter().info("waitForMbean Still probing for service");
               return false;
             } else {      
               final ObjectName cacheServerMBeanName = service.getCacheServerMBeanName(port0,serverMember);                            
@@ -251,7 +247,7 @@ public void waitForListClientMbean2(){
             return "waitForMbean Probing for ";
           }
         };
-        Wait.waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+        waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
       }
     }); 
     
@@ -276,7 +272,7 @@ public void waitForListClientMbean2(){
           public boolean done() {
             final SystemManagementService service = (SystemManagementService) ManagementService.getManagementService(getCache());
             if (service == null) {
-              Log.getLogWriter().info("waitForListClientMbean3 Still probing for service");
+              getLogWriter().info("waitForListClientMbean3 Still probing for service");
               return false;
             } else {      
               final ObjectName cacheServerMBeanName1 = service.getCacheServerMBeanName(port0,serverMember1);                            
@@ -303,13 +299,15 @@ public void waitForListClientMbean2(){
             return "waitForListClientMbean3 Probing ...";
           }
         };
-        Wait.waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+        waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
       }
     }); 
     
   }
-  
- /*public void testDescribeClientWithServers3() throws Exception {
+
+  @Ignore("disabled for unknown reason")
+  @Test
+  public void testDescribeClientWithServers3() throws Exception {
     setupSystem3();    
     String commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId + "\"" ;
     final VM server1 = Host.getHost(0).getVM(1);
@@ -350,7 +348,7 @@ public void waitForListClientMbean2(){
     
     for(String str : clientIds){
       clientId1 = str;
-      Log.getLogWriter().info("testDescribeClientWithServers clientIds for server1 ="+str);
+      getLogWriter().info("testDescribeClientWithServers clientIds for server1 ="+str);
     }
     
     final DistributedMember serverMember2 = getMember(server2);
@@ -373,21 +371,21 @@ public void waitForListClientMbean2(){
     
     for(String str : clientIds2){
       clientId2 = str;
-      Log.getLogWriter().info("testDescribeClientWithServers clientIds for server2 ="+str);
+      getLogWriter().info("testDescribeClientWithServers clientIds for server2 ="+str);
     }
     
     
     commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId1 + "\"" ;
     
-    Log.getLogWriter().info("testDescribeClientWithServers commandStr clientId1 ="+commandString);    
+    getLogWriter().info("testDescribeClientWithServers commandStr clientId1 ="+commandString);    
     
     
     CommandResult commandResultForClient1 = executeCommand(commandString);
-    Log.getLogWriter().info("testDescribeClientWithServers commandStr clientId1="+commandResultForClient1);    
+    getLogWriter().info("testDescribeClientWithServers commandStr clientId1="+commandResultForClient1);    
     
     
     String resultAsString = commandResultToString(commandResultForClient1);
-    Log.getLogWriter().info("testDescribeClientWithServers commandStr clientId1 ="+resultAsString);   
+    getLogWriter().info("testDescribeClientWithServers commandStr clientId1 ="+resultAsString);   
     assertTrue(Status.OK.equals(commandResultForClient1.getStatus()));
     
     verifyClientStats(commandResultForClient1, serverName1);
@@ -396,15 +394,15 @@ public void waitForListClientMbean2(){
     
     commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId2 + "\"" ;
     
-    Log.getLogWriter().info("testDescribeClientWithServers commandStr1="+commandString);    
+    getLogWriter().info("testDescribeClientWithServers commandStr1="+commandString);    
     
     
     CommandResult commandResultForClient2 = executeCommand(commandString);
-    Log.getLogWriter().info("testDescribeClientWithServers commandResult1="+commandResultForClient2);    
+    getLogWriter().info("testDescribeClientWithServers commandResult1="+commandResultForClient2);    
     
     
     resultAsString = commandResultToString(commandResultForClient2);
-    Log.getLogWriter().info("testDescribeClientWithServers resultAsString1="+resultAsString);   
+    getLogWriter().info("testDescribeClientWithServers resultAsString1="+resultAsString);   
     assertTrue(Status.OK.equals(commandResultForClient2.getStatus()));
     
     verifyClientStats(commandResultForClient2, serverName2);
@@ -414,15 +412,15 @@ public void waitForListClientMbean2(){
     closeCacheServer(Host.getHost(0).getVM(3));
     closeCacheServer(Host.getHost(0).getVM(1));
   
-  } */
+  }
  
-public void verifyClientStats(CommandResult commandResultForClient, String serverName){
+ public void verifyClientStats(CommandResult commandResultForClient, String serverName){
    CompositeResultData resultData = (CompositeResultData) commandResultForClient.getResultData();
    SectionResultData section =resultData.retrieveSection("InfoSection");
    assertNotNull(section);    
    for(int i = 0 ; i < 1 ; i++){
      TabularResultData tableRsultData = section.retrieveTableByIndex(i);
-     Log.getLogWriter().info("testDescribeClientWithServers getHeader="+tableRsultData.getHeader());
+     getLogWriter().info("testDescribeClientWithServers getHeader="+tableRsultData.getHeader());
      assertNotNull(tableRsultData);
      
      List<String> minConn = tableRsultData.retrieveAllValues(CliStrings.DESCRIBE_CLIENT_MIN_CONN);
@@ -431,7 +429,7 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
      List<String> numCqs = tableRsultData.retrieveAllValues(CliStrings.DESCRIBE_CLIENT_CQs);
      
      
-     Log.getLogWriter().info("testDescribeClientWithServers getHeader numCqs ="+ numCqs);
+     getLogWriter().info("testDescribeClientWithServers getHeader numCqs ="+ numCqs);
      
      assertTrue(minConn.contains("1"));
      assertTrue(maxConn.contains("-1"));
@@ -458,15 +456,17 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
      
    }
  }
-  
-  public void disabled_testDescribeClient() throws Exception {
+
+  @Ignore("disabled for unknown reason")
+  @Test
+  public void testDescribeClient() throws Exception {
     setupSystem();
     
-    Log.getLogWriter().info("testDescribeClient clientId="+clientId);    
+    getLogWriter().info("testDescribeClient clientId="+clientId);    
     assertNotNull(clientId);
     
     String commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId + "\"" ;
-    Log.getLogWriter().info("testDescribeClient commandStr="+commandString);
+    getLogWriter().info("testDescribeClient commandStr="+commandString);
     
     final VM server1 = Host.getHost(0).getVM(1);
     String serverName = (String) server1.invoke(new SerializableCallable(){
@@ -480,11 +480,11 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     
     
     CommandResult commandResult = executeCommand(commandString);
-    Log.getLogWriter().info("testDescribeClient commandResult="+commandResult);    
+    getLogWriter().info("testDescribeClient commandResult="+commandResult);    
     
     
     String resultAsString = commandResultToString(commandResult);
-    Log.getLogWriter().info("testDescribeClient resultAsString="+resultAsString);   
+    getLogWriter().info("testDescribeClient resultAsString="+resultAsString);   
     assertTrue(Status.OK.equals(commandResult.getStatus()));
     
     CompositeResultData resultData = (CompositeResultData) commandResult.getResultData();
@@ -527,13 +527,14 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     closeCacheServer(Host.getHost(0).getVM(3));
     
     
-  } 
-  
+  }
+
+  @Test
   public void testDescribeClientWithServers() throws Exception {
     setupSystem2();
     
     String commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId + "\"" ;
-    Log.getLogWriter().info("testDescribeClientWithServers commandStr="+commandString);    
+    getLogWriter().info("testDescribeClientWithServers commandStr="+commandString);    
     
     
     final VM server1 = Host.getHost(0).getVM(1);
@@ -547,11 +548,11 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     
     
     CommandResult commandResult = executeCommand(commandString);
-    Log.getLogWriter().info("testDescribeClientWithServers commandResult="+commandResult);    
+    getLogWriter().info("testDescribeClientWithServers commandResult="+commandResult);    
     
     
     String resultAsString = commandResultToString(commandResult);
-    Log.getLogWriter().info("testDescribeClientWithServers resultAsString="+resultAsString);   
+    getLogWriter().info("testDescribeClientWithServers resultAsString="+resultAsString);   
     assertTrue(Status.OK.equals(commandResult.getStatus()));
     
     CompositeResultData resultData = (CompositeResultData) commandResult.getResultData();
@@ -592,9 +593,9 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     closeNonDurableClient(Host.getHost(0).getVM(3));
     closeCacheServer(Host.getHost(0).getVM(1));
   
-  } 
-  
-  
+  }
+
+  @Test
   public void testListClient() throws Exception {
     setupSystemForListClient();
 
@@ -602,7 +603,7 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     final VM manager = Host.getHost(0).getVM(0);   
     
     String commandString = CliStrings.LIST_CLIENTS ;
-    Log.getLogWriter().info("testListClient commandStr="+commandString);
+    getLogWriter().info("testListClient commandStr="+commandString);
     
     waitForListClientMbean();  
     
@@ -632,11 +633,11 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     });
     
     CommandResult commandResult = executeCommand(commandString);
-    Log.getLogWriter().info("testListClient commandResult="+commandResult);    
+    getLogWriter().info("testListClient commandResult="+commandResult);    
     
     
     String resultAsString = commandResultToString(commandResult);
-    Log.getLogWriter().info("testListClient resultAsString="+resultAsString);   
+    getLogWriter().info("testListClient resultAsString="+resultAsString);   
     assertTrue(Status.OK.equals(commandResult.getStatus()));
     
     
@@ -650,14 +651,14 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     List<String> clientNames = tableRsultData.retrieveAllValues(CliStrings.LIST_CLIENT_COLUMN_Clients);
     
     
-    Log.getLogWriter().info("testListClients serverNames : " + serverNames);    
-    Log.getLogWriter().info("testListClients clientNames : " + clientNames);  
+    getLogWriter().info("testListClients serverNames : " + serverNames);    
+    getLogWriter().info("testListClients clientNames : " + clientNames);  
     assertEquals(2, serverNames.size());
     assertEquals(2, clientNames.size());    
     assertTrue(clientNames.contains(clientIds[0]));
     assertTrue(clientNames.contains(clientIds[1]));
     serverName = serverName.replace(":", "-");
-    Log.getLogWriter().info("testListClients serverName : " + serverName);
+    getLogWriter().info("testListClients serverName : " + serverName);
     for(String str : serverNames){
       assertTrue(str.contains(serverName));
     }    
@@ -666,17 +667,17 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     closeCacheServer(Host.getHost(0).getVM(3));
     
     
-  } 
-  
-  
- public void testListClientForServers() throws Exception {
+  }
+
+  @Test
+  public void testListClientForServers() throws Exception {
     setupSystem3();
 
     
     final VM manager = Host.getHost(0).getVM(0);   
     
     String commandString = CliStrings.LIST_CLIENTS ;
-    Log.getLogWriter().info("testListClientForServers commandStr="+commandString);
+    getLogWriter().info("testListClientForServers commandStr="+commandString);
     
     
     
@@ -716,11 +717,11 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     });
     
     CommandResult commandResult = executeCommand(commandString);
-    Log.getLogWriter().info("testListClientForServers commandResult="+commandResult);    
+    getLogWriter().info("testListClientForServers commandResult="+commandResult);    
     
     
     String resultAsString = commandResultToString(commandResult);
-    Log.getLogWriter().info("testListClientForServers resultAsString="+resultAsString);   
+    getLogWriter().info("testListClientForServers resultAsString="+resultAsString);   
     assertTrue(Status.OK.equals(commandResult.getStatus()));
     
     
@@ -737,10 +738,10 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
     serverName2 = serverName2.replace(":", "-");
     
     
-    Log.getLogWriter().info("testListClientForServers serverNames : " + serverNames);
-    Log.getLogWriter().info("testListClientForServers serverName1 : " + serverName1);
-    Log.getLogWriter().info("testListClientForServers serverName2 : " + serverName2);
-    Log.getLogWriter().info("testListClientForServers clientNames : " + clientNames);
+    getLogWriter().info("testListClientForServers serverNames : " + serverNames);
+    getLogWriter().info("testListClientForServers serverName1 : " + serverName1);
+    getLogWriter().info("testListClientForServers serverName2 : " + serverName2);
+    getLogWriter().info("testListClientForServers clientNames : " + clientNames);
     
     for(String client : clientIds){
       assertTrue(clientNames.contains(client));
@@ -983,7 +984,7 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
           getSystem(props);
           
           final ClientCacheFactory ccf = new ClientCacheFactory(props);
-          ccf.addPoolServer(NetworkUtils.getServerHostName(server.getHost()), port);
+          ccf.addPoolServer(getServerHostName(server.getHost()), port);
           ccf.setPoolSubscriptionEnabled(true);
           ccf.setPoolPingInterval(1);
           ccf.setPoolStatisticInterval(1);
@@ -1003,7 +1004,7 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
         }else{
           String poolName = "new_pool_" + System.currentTimeMillis();
           try{                      
-            PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(NetworkUtils.getServerHostName(server.getHost()), port)
+            PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(getServerHostName(server.getHost()), port)
               .setThreadLocalConnections(true)
               .setMinConnections(1)
               .setSubscriptionEnabled(true)
@@ -1056,7 +1057,7 @@ public void verifyClientStats(CommandResult commandResultForClient, String serve
 
   protected Properties getServerProperties() {
     Properties p = new Properties();
-    p.setProperty(DistributionConfig.LOCATORS_NAME, "localhost["+DistributedTestUtils.getDUnitLocatorPort()+"]");
+    p.setProperty(DistributionConfig.LOCATORS_NAME, "localhost["+getDUnitLocatorPort()+"]");
     return p;
   }
   
@@ -1077,14 +1078,14 @@ public void waitForNonSubCliMBean(){
             try {         
               final SystemManagementService service = (SystemManagementService) ManagementService.getManagementService(getCache());
               if (service == null) {
-                Log.getLogWriter().info("waitForNonSubScribedClientMBean Still probing for service");
+                getLogWriter().info("waitForNonSubScribedClientMBean Still probing for service");
                 return false;
               } else {      
-                Log.getLogWriter().info("waitForNonSubScribedClientMBean 1");
+                getLogWriter().info("waitForNonSubScribedClientMBean 1");
                 final ObjectName cacheServerMBeanName = service.getCacheServerMBeanName(port0,serverMember);
-                Log.getLogWriter().info("waitForNonSubScribedClientMBean 2 cacheServerMBeanName " + cacheServerMBeanName);
+                getLogWriter().info("waitForNonSubScribedClientMBean 2 cacheServerMBeanName " + cacheServerMBeanName);
                 CacheServerMXBean bean = service.getMBeanProxy(cacheServerMBeanName, CacheServerMXBean.class);
-                Log.getLogWriter().info("waitForNonSubScribedClientMBean 2 bean " + bean);
+                getLogWriter().info("waitForNonSubScribedClientMBean 2 bean " + bean);
                 if(bean.getClientIds().length > 0){
                   return true;
                 }               
@@ -1100,7 +1101,7 @@ public void waitForNonSubCliMBean(){
             return "waitForNonSubScribedClientMBean Probing for ";
           }
         };
-        Wait.waitForCriterion(waitForMaangerMBean, 5* 60 * 1000, 2000, true);
+        waitForCriterion(waitForMaangerMBean, 5* 60 * 1000, 2000, true);
       }
     }); 
     
@@ -1125,14 +1126,14 @@ public void waitForMixedClients(){
             try {         
               final SystemManagementService service = (SystemManagementService) ManagementService.getManagementService(getCache());
               if (service == null) {
-                Log.getLogWriter().info("waitForMixedClients Still probing for service");
+                getLogWriter().info("waitForMixedClients Still probing for service");
                 return false;
               } else {      
-                Log.getLogWriter().info("waitForMixedClients 1");
+                getLogWriter().info("waitForMixedClients 1");
                 final ObjectName cacheServerMBeanName = service.getCacheServerMBeanName(port0,serverMember);
-                Log.getLogWriter().info("waitForMixedClients 2 cacheServerMBeanName " + cacheServerMBeanName);
+                getLogWriter().info("waitForMixedClients 2 cacheServerMBeanName " + cacheServerMBeanName);
                 CacheServerMXBean bean = service.getMBeanProxy(cacheServerMBeanName, CacheServerMXBean.class);
-                Log.getLogWriter().info("waitForMixedClients 2 bean " + bean);
+                getLogWriter().info("waitForMixedClients 2 bean " + bean);
                 if(bean.getClientIds().length > 1){
                   return true;
                 }                
@@ -1148,22 +1149,21 @@ public void waitForMixedClients(){
             return "waitForMixedClients Probing for ";
           }
         };
-        Wait.waitForCriterion(waitForMaangerMBean, 5* 60 * 1000, 2000, true);
+        waitForCriterion(waitForMaangerMBean, 5* 60 * 1000, 2000, true);
       }
     }); 
     
   }
   
-  
-  
+  @Test
   public void testDescribeClientForNonSubscribedClient() throws Exception {
     setUpNonSubscribedClient();
     
-    Log.getLogWriter().info("testDescribeClientForNonSubscribedClient clientId="+clientId);    
+    getLogWriter().info("testDescribeClientForNonSubscribedClient clientId="+clientId);    
     assertNotNull(clientId);
     
     String commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientId + "\"" ;
-    Log.getLogWriter().info("testDescribeClientForNonSubscribedClient commandStr="+commandString);
+    getLogWriter().info("testDescribeClientForNonSubscribedClient commandStr="+commandString);
     
     final VM server1 = Host.getHost(0).getVM(1);
     String serverName = (String) server1.invoke(new SerializableCallable(){
@@ -1176,11 +1176,11 @@ public void waitForMixedClients(){
     
     
     CommandResult commandResult = executeCommand(commandString);
-    Log.getLogWriter().info("testDescribeClientForNonSubscribedClient commandResult="+commandResult);    
+    getLogWriter().info("testDescribeClientForNonSubscribedClient commandResult="+commandResult);    
     
     
     String resultAsString = commandResultToString(commandResult);
-    Log.getLogWriter().info("testDescribeClientForNonSubscribedClient resultAsString="+resultAsString);   
+    getLogWriter().info("testDescribeClientForNonSubscribedClient resultAsString="+resultAsString);   
     assertTrue(Status.OK.equals(commandResult.getStatus()));
     
     CompositeResultData resultData = (CompositeResultData) commandResult.getResultData();
@@ -1228,8 +1228,9 @@ public void waitForMixedClients(){
     closeCacheServer(Host.getHost(0).getVM(3));
     
     
-  }  
-  
+  }
+
+  @Test
   public void testDescribeMixClientWithServers() throws Exception {
     String[] clientIds = setupSystemWithSubAndNonSubClient();    
     
@@ -1243,13 +1244,13 @@ public void waitForMixedClients(){
     });
     
     String commandString = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientIds[0] + "\"" ;
-    Log.getLogWriter().info("testDescribeMixClientWithServers commandStr="+commandString);
+    getLogWriter().info("testDescribeMixClientWithServers commandStr="+commandString);
     
     
     executeAndVerifyResultsForMixedClients(commandString, serverName );    
     
     String commandString2 = CliStrings.DESCRIBE_CLIENT + " --" + CliStrings.DESCRIBE_CLIENT__ID + "=\""+ clientIds[1] + "\"" ;
-    Log.getLogWriter().info("testDescribeMixClientWithServers commandString2="+commandString2);   
+    getLogWriter().info("testDescribeMixClientWithServers commandString2="+commandString2);   
     
     
     executeAndVerifyResultsForMixedClients(commandString2,serverName );
@@ -1262,11 +1263,11 @@ public void waitForMixedClients(){
   
 void executeAndVerifyResultsForMixedClients(String commandString, String serverName){
   CommandResult commandResult = executeCommand(commandString);
-  Log.getLogWriter().info("testDescribeMixClientWithServers commandResult="+commandResult);    
+  getLogWriter().info("testDescribeMixClientWithServers commandResult="+commandResult);    
   
   
   String resultAsString = commandResultToString(commandResult);
-  Log.getLogWriter().info("testDescribeMixClientWithServers resultAsString="+resultAsString);
+  getLogWriter().info("testDescribeMixClientWithServers resultAsString="+resultAsString);
   
   
   assertTrue(Status.OK.equals(commandResult.getStatus()));
@@ -1396,7 +1397,7 @@ private void setUpNonSubscribedClient() throws Exception {
           getSystem(props);
           
           final ClientCacheFactory ccf = new ClientCacheFactory(props);
-          ccf.addPoolServer(NetworkUtils.getServerHostName(server.getHost()), port);
+          ccf.addPoolServer(getServerHostName(server.getHost()), port);
           ccf.setPoolSubscriptionEnabled(false);
           ccf.setPoolPingInterval(1);
           ccf.setPoolStatisticInterval(1);
@@ -1416,7 +1417,7 @@ private void setUpNonSubscribedClient() throws Exception {
         }else{
           String poolName = "new_pool_" + System.currentTimeMillis();
           try{                      
-            PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(NetworkUtils.getServerHostName(server.getHost()), port)
+            PoolImpl p = (PoolImpl) PoolManager.createFactory().addServer(getServerHostName(server.getHost()), port)
               .setThreadLocalConnections(true)
               .setMinConnections(1)
               .setSubscriptionEnabled(false)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
index 0a1f433..b35538d 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DurableClientCommandsDUnitTest.java
@@ -16,13 +16,21 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.DistributedTestUtils.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.NetworkUtils.*;
+
 import java.util.Properties;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
@@ -44,15 +52,12 @@ import com.gemstone.gemfire.management.cli.Result.Status;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
 import com.gemstone.gemfire.test.dunit.Host;
-import com.gemstone.gemfire.test.dunit.LogWriterUtils;
-import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
-
-
+@Category(DistributedTest.class)
 public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
 
   private static final long serialVersionUID = 1L;
@@ -61,11 +66,8 @@ public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
   final String cq2 = "cq2";
   final String cq3 = "cq3";
   final String clientName = "dc1";
-  
-  public DurableClientCommandsDUnitTest(String name) {
-    super(name);
-  }
- 
+
+  @Test
   public void testListDurableClientCqs() throws Exception {
     setupSystem();
     setupCqs();
@@ -98,6 +100,7 @@ public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
     assertTrue(resultAsString.contains(errorMessage));
   }
   
+  @Test
   public void testCloseDurableClients() throws Exception {
     setupSystem();
     setupCqs();
@@ -127,8 +130,8 @@ public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
     String errorMessage = CliStrings.format(CliStrings.NO_CLIENT_FOUND_WITH_CLIENT_ID, clientName);
     assertTrue(resultAsString.contains(errorMessage));
   }
-  
-  
+
+  @Test
   public void testCloseDurableCQ() throws Exception{
     setupSystem();
     setupCqs();
@@ -156,17 +159,8 @@ public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
     assertTrue(Status.ERROR.equals(commandResult.getStatus()));
     
   }
-    
-//  public void testRepeat() throws Exception {
-//    long endTime = System.currentTimeMillis() + (75 * 60000);
-//    while (endTime > System.currentTimeMillis()) {
-//      testCountSubscriptionQueueSize();
-//      tearDown();
-//      setUp();
-//    }
-//    testCountSubscriptionQueueSize();
-//  }
-//  
+
+  @Test
   public void testCountSubscriptionQueueSize() throws Exception {
     setupSystem();
     setupCqs();
@@ -250,7 +244,7 @@ public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
   }
   
   private void writeToLog(String text, String resultAsString) {
-    LogWriterUtils.getLogWriter().info(getUniqueName() + ": " + text + "\n" + resultAsString);
+    getLogWriter().info(getUniqueName() + ": " + text + "\n" + resultAsString);
   }
   
   private void setupSystem() throws Exception {
@@ -360,7 +354,7 @@ public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
         getSystem(props);
         
         final ClientCacheFactory ccf = new ClientCacheFactory(props);
-        ccf.addPoolServer(NetworkUtils.getServerHostName(server.getHost()), port);
+        ccf.addPoolServer(getServerHostName(server.getHost()), port);
         ccf.setPoolSubscriptionEnabled(true);
         
         ClientCache cache = (ClientCache)getClientCache(ccf);
@@ -368,10 +362,9 @@ public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
     });
   }
   
-  /* Does few puts on the region on the server
-   * 
+  /**
+   * Does few puts on the region on the server
    */
-  
   private void doPuts(final String regionName, VM server) {
     server.invoke(new SerializableCallable() {
       public Object call() throws Exception {
@@ -423,7 +416,7 @@ public class DurableClientCommandsDUnitTest extends CliCommandTestBase {
 
   protected Properties getServerProperties() {
     Properties p = new Properties();
-    p.setProperty(DistributionConfig.LOCATORS_NAME, "localhost["+DistributedTestUtils.getDUnitLocatorPort()+"]");
+    p.setProperty(DistributionConfig.LOCATORS_NAME, "localhost["+getDUnitLocatorPort()+"]");
     return p;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java
index e2950c2..f268c8f 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthorizationTwoDUnitTest.java
@@ -19,11 +19,13 @@ package com.gemstone.gemfire.security;
 import static com.gemstone.gemfire.security.SecurityTestUtils.*;
 import static com.gemstone.gemfire.test.dunit.IgnoredException.*;
 
-import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
+
 /**
  * Tests for authorization from client to server. This tests for authorization
  * of all operations with both valid and invalid credentials/modules with
@@ -35,7 +37,7 @@ import org.junit.experimental.categories.Category;
  * 
  * @since 5.5
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class ClientAuthorizationTwoDUnitTest extends ClientAuthorizationTestCase {
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java
index 0e8e57f..fe0e4c3 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientAuthzObjectModDUnitTest.java
@@ -26,6 +26,9 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.DataSerializable;
 import com.gemstone.gemfire.Instantiator;
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
@@ -38,8 +41,7 @@ import com.gemstone.gemfire.security.generator.DummyCredentialGenerator;
 import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 
 /**
  * Tests for authorization callback that modify objects and callbacks from
@@ -57,7 +59,7 @@ import org.junit.experimental.categories.Category;
  * 
  * @since 5.5
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class ClientAuthzObjectModDUnitTest extends ClientAuthorizationTestCase {
 
   private static final String preAccessor = FilterPreAuthorization.class.getName() + ".create";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java
index dfa89f9..e3e7886 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientCQPostAuthorizationDUnitTest.java
@@ -29,6 +29,9 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 import com.gemstone.gemfire.cache.query.CqAttributes;
@@ -50,13 +53,12 @@ import com.gemstone.gemfire.security.generator.CredentialGenerator;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 
 /**
  * This is for multiuser-authentication
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class ClientCQPostAuthorizationDUnitTest extends ClientAuthorizationTestCase {
 
   private Map<String, String> cqNameToQueryStrings = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java
index ce03ac6..5db15f0 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/security/ClientPostAuthorizationDUnitTest.java
@@ -26,15 +26,18 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 import com.gemstone.gemfire.security.generator.AuthzCredentialGenerator;
 import com.gemstone.gemfire.security.generator.CredentialGenerator;
 import com.gemstone.gemfire.test.junit.Retry;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 import com.gemstone.gemfire.test.junit.rules.RetryRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 /**
  * Tests for authorization from client to server. This tests for authorization
@@ -43,14 +46,14 @@ import org.junit.experimental.categories.Category;
  * 
  * @since 5.5
  */
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class, FlakyTest.class}) // GEODE-693, GEODE-1009: getRandomAvailablePort
 public class ClientPostAuthorizationDUnitTest extends ClientAuthorizationTestCase {
 
   @Rule
   public RetryRule retryRule = new RetryRule();
 
   @Test
-  @Retry(2)
+  @Retry(2) // GEODE-693: getRandomAvailablePort
   public void testAllPostOps() throws Exception {
     OperationWithAction[] allOps = allOpsForTestAllPostOps();
 
@@ -112,6 +115,7 @@ public class ClientPostAuthorizationDUnitTest extends ClientAuthorizationTestCas
   }
 
   @Test
+  @Retry(2) // GEODE-1009: getRandomAvailablePort
   public void testAllOpsNotifications() throws Exception {
     OperationWithAction[] allOps = allOpsForTestAllOpsNotifications();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserAPIDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserAPIDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserAPIDUnitTest.java
index 9e04f5f..9bbceff 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserAPIDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserAPIDUnitTest.java
@@ -26,6 +26,9 @@ import java.util.Properties;
 import javax.net.ssl.SSLException;
 import javax.net.ssl.SSLHandshakeException;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.Pool;
 import com.gemstone.gemfire.cache.execute.FunctionService;
@@ -38,10 +41,9 @@ import com.gemstone.gemfire.internal.cache.PoolManagerImpl;
 import com.gemstone.gemfire.security.generator.CredentialGenerator;
 import com.gemstone.gemfire.security.generator.DummyCredentialGenerator;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class MultiUserAPIDUnitTest extends ClientAuthorizationTestCase {
 
   private static final String[] serverIgnoredExceptions = {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserDurableCQAuthzDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserDurableCQAuthzDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserDurableCQAuthzDUnitTest.java
index 632a997..3a97b7c 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserDurableCQAuthzDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/security/MultiUserDurableCQAuthzDUnitTest.java
@@ -27,6 +27,9 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.operations.OperationContext.OperationCode;
 import com.gemstone.gemfire.cache.query.CqAttributes;
@@ -45,10 +48,9 @@ import com.gemstone.gemfire.security.generator.AuthzCredentialGenerator;
 import com.gemstone.gemfire.security.generator.CredentialGenerator;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
 
-@Category(DistributedTest.class)
+@Category({ DistributedTest.class, SecurityTest.class })
 public class MultiUserDurableCQAuthzDUnitTest extends ClientAuthorizationTestCase {
 
   private final Map<String, String> cqNameToQueryStrings = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/FlakyTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/FlakyTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/FlakyTest.java
new file mode 100644
index 0000000..b1e9b18
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/FlakyTest.java
@@ -0,0 +1,24 @@
+/*
+ * 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.test.junit.categories;
+
+/**
+ * JUnit Test Category that specifies a flickering test that fails
+ * intermittently.
+ */
+public interface FlakyTest {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/SecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/SecurityTest.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/SecurityTest.java
new file mode 100644
index 0000000..3af739e
--- /dev/null
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/categories/SecurityTest.java
@@ -0,0 +1,23 @@
+/*
+ * 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.test.junit.categories;
+
+/**
+ * JUnit Test Category that specifies a test involving security.
+ */
+public class SecurityTest {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java
index 8535fa5..7955d97 100755
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/RetryRule.java
@@ -79,7 +79,7 @@ public class RetryRule implements TestRule, Serializable {
     return this.implementation.apply(base, description);
   }
 
-  protected abstract class AbstractRetryRule implements TestRule {
+  protected abstract class AbstractRetryRule implements TestRule, Serializable {
     protected AbstractRetryRule() {
     }
     protected void evaluate(final Statement base, final Description description, final int retryCount) throws Throwable {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryOne.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryOne.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryOne.java
new file mode 100644
index 0000000..8d3549b
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryOne.java
@@ -0,0 +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.
+ */
+package com.gemstone.gemfire.test.junit.categories;
+
+public interface CategoryOne {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryTest.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryTest.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryTest.java
new file mode 100644
index 0000000..be342c1
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryTest.java
@@ -0,0 +1,169 @@
+/*
+ * 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.test.junit.categories;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Categories;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.Result;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+import com.gemstone.gemfire.test.junit.rules.TestRunner;
+
+@Category(UnitTest.class)
+public class CategoryTest {
+
+  private static boolean executedClassOneMethodNone; // 1
+  private static boolean executedClassOneMethodTwo; // 2
+  private static boolean executedClassTwoMethodTwo; // 3
+  private static boolean executedClassNoneMethodOne; // 4
+  private static boolean executedClassNoneMethodTwo; // 5
+  private static boolean executedClassTwoMethodOne; // 6
+  private static boolean executedClassOneMethodOne; // 7
+  private static boolean executedClassOneAndTwoMethodNone; // 8
+  private static boolean executedClassNoneMethodOneAndTwo; // 9
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    executedClassOneMethodNone = false;
+    executedClassOneMethodTwo = false;
+    executedClassTwoMethodTwo = false;
+    executedClassNoneMethodOne = false;
+    executedClassNoneMethodTwo = false;
+    executedClassTwoMethodOne = false;
+    executedClassOneMethodOne = false;
+    executedClassOneAndTwoMethodNone = false;
+    executedClassNoneMethodOneAndTwo = false;
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+  }
+
+  @Test
+  public void allTestsWithCategoryOneShouldBeExecuted() {
+    Result result = TestRunner.runTest(CategoryTestSuite.class);
+
+    assertThat(result.wasSuccessful()).isTrue();
+    assertThat(executedClassOneMethodNone).isTrue();
+    assertThat(executedClassOneMethodTwo).isTrue();
+    assertThat(executedClassTwoMethodTwo).isFalse();
+    assertThat(executedClassNoneMethodOne).isTrue();
+    assertThat(executedClassNoneMethodTwo).isFalse();
+    assertThat(executedClassTwoMethodOne).isTrue();
+    assertThat(executedClassOneMethodOne).isTrue();
+    assertThat(executedClassOneAndTwoMethodNone).isTrue();
+    assertThat(executedClassNoneMethodOneAndTwo).isTrue();
+  }
+
+  @Category(CategoryOne.class)
+  public static class ClassOneMethodNone { // 1
+    @Test
+    public void test() {
+      executedClassOneMethodNone = true;
+    }
+  }
+
+  @Category(CategoryOne.class)
+  public static class ClassOneMethodTwo { // 2
+    @Category(CategoryTwo.class)
+    @Test
+    public void test() {
+      executedClassOneMethodTwo = true;
+    }
+  }
+
+  @Category(CategoryTwo.class)
+  public static class ClassTwoMethodTwo { // 3
+    @Category(CategoryTwo.class)
+    @Test
+    public void test() {
+      executedClassTwoMethodTwo = true;
+    }
+  }
+
+  public static class ClassNoneMethodOne { // 4
+    @Category(CategoryOne.class)
+    @Test
+    public void test() {
+      executedClassNoneMethodOne = true;
+    }
+  }
+
+  public static class ClassNoneMethodTwo { // 5
+    @Category(CategoryTwo.class)
+    @Test
+    public void test() {
+      executedClassNoneMethodTwo = true;
+    }
+  }
+
+  @Category(CategoryTwo.class)
+  public static class ClassTwoMethodOne { // 6
+    @Category(CategoryOne.class)
+    @Test
+    public void test() {
+      executedClassTwoMethodOne = true;
+    }
+  }
+
+  @Category(CategoryOne.class)
+  public static class ClassOneMethodOne { // 7
+    @Category(CategoryOne.class)
+    @Test
+    public void test() {
+      executedClassOneMethodOne = true;
+    }
+  }
+
+  @Category({ CategoryOne.class, CategoryTwo.class })
+  public static class ClassOneAndTwoMethodNone { // 8
+    @Test
+    public void test() {
+      executedClassOneAndTwoMethodNone = true;
+    }
+  }
+
+  public static class ClassNoneMethodOneAndTwo { // 9
+    @Category({ CategoryOne.class, CategoryTwo.class })
+    @Test
+    public void test() {
+      executedClassNoneMethodOneAndTwo = true;
+    }
+  }
+
+  @RunWith(Categories.class)
+  @Categories.IncludeCategory(CategoryOne.class)
+  @Suite.SuiteClasses({
+          ClassOneMethodNone.class, // 1
+          ClassOneMethodTwo.class, // 2
+          ClassTwoMethodTwo.class, // 3
+          ClassNoneMethodOne.class, // 4
+          ClassNoneMethodTwo.class, // 5
+          ClassTwoMethodOne.class, // 6
+          ClassOneMethodOne.class, // 7
+          ClassOneAndTwoMethodNone.class, // 8
+          ClassNoneMethodOneAndTwo.class // 9
+  })
+  public static class CategoryTestSuite {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryTwo.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryTwo.java b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryTwo.java
new file mode 100644
index 0000000..4a88339
--- /dev/null
+++ b/geode-junit/src/test/java/com/gemstone/gemfire/test/junit/categories/CategoryTwo.java
@@ -0,0 +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.
+ */
+package com.gemstone.gemfire.test.junit.categories;
+
+public interface CategoryTwo {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java
index 9bfce05..719fd92 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WANCommandTestBase.java
@@ -16,13 +16,14 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 import java.util.Set;
-
 import javax.management.remote.JMXConnectorServer;
 
 import com.gemstone.gemfire.cache.Cache;
@@ -44,13 +45,13 @@ import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySender;
 import com.gemstone.gemfire.management.ManagementService;
 import com.gemstone.gemfire.management.internal.cli.commands.CliCommandTestBase;
+import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
 import com.gemstone.gemfire.test.dunit.Invoke;
-import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 
-public class WANCommandTestBase extends CliCommandTestBase{
+public class WANCommandTestBase extends CliCommandTestBase {
 
   static Cache cache;
   private JMXConnectorServer jmxConnectorServer;
@@ -67,10 +68,6 @@ public class WANCommandTestBase extends CliCommandTestBase{
   static VM vm6;
   static VM vm7;
 
-  public WANCommandTestBase(String name) {
-    super(name);
-  }
-
   @Override
   public final void postSetUp() throws Exception {
     final Host host = Host.getHost(0);
@@ -86,7 +83,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static Integer createFirstLocatorWithDSId(int dsId) {
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
     Properties props = test.getDistributedSystemProperties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME,"0");
@@ -99,7 +96,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static Integer createFirstRemoteLocator(int dsId, int remoteLocPort) {
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
     Properties props = test.getDistributedSystemProperties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME,"0");
@@ -112,7 +109,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static void createCache(Integer locPort){
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     Properties props = test.getDistributedSystemProperties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort + "]");
@@ -121,7 +118,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static void createCacheWithGroups(Integer locPort, String groups){
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     Properties props = test.getDistributedSystemProperties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort + "]");
@@ -214,7 +211,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static int createAndStartReceiver(int locPort) {
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     Properties props = test.getDistributedSystemProperties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort
@@ -238,7 +235,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static int createReceiver(int locPort) {
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     Properties props = test.getDistributedSystemProperties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort
@@ -256,7 +253,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static int createReceiverWithGroup(int locPort, String groups) {
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     Properties props =  test.getDistributedSystemProperties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort
@@ -275,7 +272,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static void startReceiver() {
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     try {
       Set<GatewayReceiver> receivers = cache.getGatewayReceivers();
       for (GatewayReceiver receiver : receivers) {
@@ -288,7 +285,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static void stopReceiver() {
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     Set<GatewayReceiver> receivers = cache.getGatewayReceivers();
     for (GatewayReceiver receiver : receivers) {
       receiver.stop();
@@ -296,7 +293,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static int createAndStartReceiverWithGroup(int locPort, String groups) {
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     Properties props = test.getDistributedSystemProperties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
     props.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort
@@ -444,7 +441,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
   }
 
   public static void verifyReceiverState(boolean isRunning) {
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     Set<GatewayReceiver> receivers = cache.getGatewayReceivers();
     for (GatewayReceiver receiver : receivers) {
       assertEquals(isRunning, receiver.isRunning());
@@ -455,7 +452,7 @@ public class WANCommandTestBase extends CliCommandTestBase{
       int startPort, int endPort, String bindAddress, int maxTimeBetweenPings,
       int socketBufferSize, List<String> expectedGatewayTransportFilters) {
 
-    WANCommandTestBase test = new WANCommandTestBase(getTestMethodName());
+    WANCommandTestBase test = new WANCommandTestBase();
     Set<GatewayReceiver> receivers = cache.getGatewayReceivers();
     assertEquals("Number of receivers is incorrect", 1, receivers.size());
     for (GatewayReceiver receiver : receivers) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewayReceiverDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewayReceiverDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewayReceiverDUnitTest.java
index 1122059..d9a551b 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewayReceiverDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewayReceiverDUnitTest.java
@@ -16,12 +16,16 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.wan.GatewayReceiver;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -31,22 +35,20 @@ import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * DUnit tests for 'create gateway-receiver' command.
  */
-public class WanCommandCreateGatewayReceiverDUnitTest extends
-    WANCommandTestBase {
+@Category(DistributedTest.class)
+public class WanCommandCreateGatewayReceiverDUnitTest extends WANCommandTestBase {
 
   private static final long serialVersionUID = 1L;
   
-  public WanCommandCreateGatewayReceiverDUnitTest(String name) {
-    super(name);
-  }
-  
   /**
    * GatewayReceiver with all default attributes
    */
+  @Test
   public void testCreateGatewayReceiverWithDefault() {
 
     VM puneLocator = Host.getLocator();
@@ -68,7 +70,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -109,6 +111,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
   /**
    * GatewayReceiver with given attributes
    */
+  @Test
   public void testCreateGatewayReceiver() {
 
     VM puneLocator = Host.getLocator();
@@ -136,7 +139,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -165,6 +168,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
   /**
    * GatewayReceiver with given attributes and a single GatewayTransportFilter.
    */
+  @Test
   public void testCreateGatewayReceiverWithGatewayTransportFilter() {
 
     VM puneLocator = Host.getLocator();
@@ -193,7 +197,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -225,6 +229,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
   /**
    * GatewayReceiver with given attributes and multiple GatewayTransportFilters.
    */
+  @Test
   public void testCreateGatewayReceiverWithMultipleGatewayTransportFilters() {
 
     VM puneLocator = Host.getLocator();
@@ -254,7 +259,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -288,6 +293,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
    * GatewayReceiver with given attributes.
    * Error scenario where startPort is greater than endPort.
    */
+  @Test
   public void testCreateGatewayReceiver_Error() {
 
     VM puneLocator = Host.getLocator();
@@ -314,7 +320,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -336,6 +342,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
   /**
    * GatewayReceiver with given attributes on the given member.
    */
+  @Test
   public void testCreateGatewayReceiver_onMember() {
 
     VM puneLocator = Host.getLocator();
@@ -366,7 +373,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -391,6 +398,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
   /**
    * GatewayReceiver with given attributes on multiple members.
    */
+  @Test
   public void testCreateGatewayReceiver_onMultipleMembers() {
 
     VM puneLocator = Host.getLocator();
@@ -422,7 +430,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -449,6 +457,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
   /**
    * GatewayReceiver with given attributes on the given group.
    */
+  @Test
   public void testCreateGatewayReceiver_onGroup() {
 
     VM puneLocator = Host.getLocator();
@@ -477,7 +486,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -507,6 +516,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
    * GatewayReceiver with given attributes on the given group.
    * Only 2 of 3 members are part of the group.
    */
+  @Test
   public void testCreateGatewayReceiver_onGroup_Scenario2() {
 
     VM puneLocator = Host.getLocator();
@@ -535,7 +545,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -562,6 +572,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
   /**
    * GatewayReceiver with given attributes on multiple groups.
    */
+  @Test
   public void testCreateGatewayReceiver_onMultipleGroups() {
 
     VM puneLocator = Host.getLocator();
@@ -590,7 +601,7 @@ public class WanCommandCreateGatewayReceiverDUnitTest extends
     CommandResult cmdResult = executeCommand(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewayReceiver stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2d5969e/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java
index 2c99ad3..d5c4b75 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/wancommand/WanCommandCreateGatewaySenderDUnitTest.java
@@ -16,14 +16,18 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.wancommand;
 
-import hydra.Log;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
-import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.wan.GatewaySender;
+import com.gemstone.gemfire.cache.wan.GatewaySender.OrderPolicy;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.cache.wan.GatewaySenderException;
@@ -32,15 +36,13 @@ import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+@Category(DistributedTest.class)
 public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
   
   private static final long serialVersionUID = 1L;
   
-  public WanCommandCreateGatewaySenderDUnitTest(String name) {
-    super(name);
-  }
-
   private CommandResult executeCommandWithIgnoredExceptions(String command) {
     final IgnoredException exln = IgnoredException.addIgnoredException("Could not connect");
     try {
@@ -55,6 +57,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
   /**
    * GatewaySender with all default attributes
    */
+  @Test
   public void testCreateGatewaySenderWithDefault() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -77,7 +80,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -103,6 +106,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
   /**
    * GatewaySender with given attribute values
    */
+  @Test
   public void testCreateGatewaySender() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -139,7 +143,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -174,6 +178,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
    * Error scenario where dispatcher threads is set to more than 1 and 
    * no order policy provided.
    */
+  @Test
   public void testCreateGatewaySender_Error() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -209,7 +214,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -226,10 +231,10 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
 
   }
 
-  
   /**
    * GatewaySender with given attribute values and event filters.
    */
+  @Test
   public void testCreateGatewaySenderWithGatewayEventFilters() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -268,7 +273,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -304,6 +309,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
   /**
    * GatewaySender with given attribute values and transport filters.
    */
+  @Test
   public void testCreateGatewaySenderWithGatewayTransportFilters() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -341,7 +347,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -376,6 +382,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
   /**
    * GatewaySender with given attribute values on given member.
    */
+  @Test
   public void testCreateGatewaySender_OnMember() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -415,7 +422,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -440,6 +447,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
   /**
    * GatewaySender with given attribute values on given group
    */
+  @Test
   public void testCreateGatewaySender_Group() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -477,7 +485,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -504,6 +512,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
    * GatewaySender with given attribute values on given group.
    * Only 2 of 3 members are part of the group.
    */
+  @Test
   public void testCreateGatewaySender_Group_Scenario2() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -541,7 +550,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -565,6 +574,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
   /**
    * Parallel GatewaySender with given attribute values
    */
+  @Test
   public void testCreateParallelGatewaySender() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -599,7 +609,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
     CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
     if (cmdResult != null) {
       String strCmdResult = commandResultToString(cmdResult);
-      Log.getLogWriter().info(
+      getLogWriter().info(
           "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
       assertEquals(Result.Status.OK, cmdResult.getStatus());
 
@@ -633,6 +643,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
    * Parallel GatewaySender with given attribute values.
    * Provide dispatcherThreads as 2 which is not valid for Parallel sender.
    */
+  @Test
   public void testCreateParallelGatewaySender_Error() {
 
     Integer punePort = (Integer) vm1.invoke(() -> WANCommandTestBase.createFirstLocatorWithDSId( 1 ));
@@ -673,7 +684,7 @@ public class WanCommandCreateGatewaySenderDUnitTest extends WANCommandTestBase {
       CommandResult cmdResult = executeCommandWithIgnoredExceptions(command);
       if (cmdResult != null) {
         String strCmdResult = commandResultToString(cmdResult);
-        Log.getLogWriter().info(
+        getLogWriter().info(
             "testCreateGatewaySender stringResult : " + strCmdResult + ">>>>");
         assertEquals(Result.Status.OK, cmdResult.getStatus());
 


[11/32] incubator-geode git commit: GEODE-1219: Updated lucene version to 6.0.0

Posted by ji...@apache.org.
GEODE-1219: Updated lucene version to 6.0.0

Modifications to code due to lucene api changes
Implemented temporary output stream in Directory and FileSystem


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 6c7a0d2a2d3c0c56828c9c0498390b505de00ea4
Parents: f473fac
Author: Jason Huynh <hu...@gmail.com>
Authored: Thu Apr 7 15:40:35 2016 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Wed Apr 13 11:13:27 2016 -0700

----------------------------------------------------------------------
 .../internal/directory/RegionDirectory.java     | 20 ++++++++++++++++----
 .../lucene/internal/filesystem/FileSystem.java  |  5 +++++
 .../repository/IndexRepositoryImpl.java         |  2 +-
 .../repository/serializer/SerializerUtil.java   | 16 ++++++++--------
 .../IndexRepositoryImplPerformanceTest.java     |  4 ++--
 gradle/dependency-versions.properties           |  2 +-
 6 files changed, 33 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c7a0d2a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java
index e25dc77..28117bd 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/RegionDirectory.java
@@ -21,7 +21,9 @@ package com.gemstone.gemfire.cache.lucene.internal.directory;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.lucene.store.BaseDirectory;
@@ -45,7 +47,7 @@ import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystem;
 public class RegionDirectory extends BaseDirectory {
 
   private final FileSystem fs;
-  
+
   /**
    * Create a region directory with a given file and chunk region. These regions
    * may be bucket regions or they may be replicated regions.
@@ -54,11 +56,13 @@ public class RegionDirectory extends BaseDirectory {
     super(new SingleInstanceLockFactory());
     fs = new FileSystem(fileRegion, chunkRegion);
   }
-  
+
   @Override
   public String[] listAll() throws IOException {
     ensureOpen();
-    return fs.listFileNames().toArray(new String[] {});
+    String[] array = fs.listFileNames().toArray(new String[]{});
+    Arrays.sort(array);
+    return array;
   }
 
   @Override
@@ -79,7 +83,15 @@ public class RegionDirectory extends BaseDirectory {
     final File file = fs.createFile(name);
     final OutputStream out = file.getOutputStream();
 
-    return new OutputStreamIndexOutput(name, out, 1000);
+    return new OutputStreamIndexOutput(name, name, out, 1000);
+  }
+
+  public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException {
+    String name = prefix + "_temp_" + UUID.randomUUID() + suffix;
+    final File file = fs.createTemporaryFile(name);
+    final OutputStream out = file.getOutputStream();
+
+    return new OutputStreamIndexOutput(name, name, out, 1000);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c7a0d2a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
index b84dc92..44513f1 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
@@ -60,6 +60,11 @@ public class FileSystem {
     // TODO unlock region ?
     return file;
   }
+
+  public File createTemporaryFile(final String name) throws IOException {
+    final File file = new File(this, name);
+    return file;
+  }
   
   public File getFile(final String name) throws FileNotFoundException {
     final File file = fileRegion.get(name);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c7a0d2a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
index a9c463e..e589ef4 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
@@ -51,7 +51,7 @@ public class IndexRepositoryImpl implements IndexRepository {
   public IndexRepositoryImpl(Region<?,?> region, IndexWriter writer, LuceneSerializer serializer) throws IOException {
     this.region = region;
     this.writer = writer;
-    searcherManager = new SearcherManager(writer, APPLY_ALL_DELETES, null);
+    searcherManager = new SearcherManager(writer, APPLY_ALL_DELETES, true, null);
     this.serializer = serializer;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c7a0d2a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/SerializerUtil.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/SerializerUtil.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/SerializerUtil.java
index 7ffc5db..0ed9d5d 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/SerializerUtil.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/serializer/SerializerUtil.java
@@ -24,11 +24,11 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.DoubleField;
+import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.Field.Store;
-import org.apache.lucene.document.FloatField;
-import org.apache.lucene.document.IntField;
-import org.apache.lucene.document.LongField;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexableField;
@@ -79,13 +79,13 @@ public class SerializerUtil {
     if(clazz == String.class) {
       doc.add(new TextField(field, (String)fieldValue, Store.NO));
     } else if (clazz == Long.class) {
-      doc.add(new LongField(field, (Long) fieldValue, Store.NO));
+      doc.add(new LongPoint(field, (Long) fieldValue));
     } else if (clazz == Integer.class) {
-      doc.add(new IntField(field, (Integer) fieldValue, Store.NO));
+      doc.add(new IntPoint(field, (Integer) fieldValue));
     } else if (clazz == Float.class) {
-      doc.add(new FloatField(field, (Float) fieldValue, Store.NO));
+      doc.add(new FloatPoint(field, (Float) fieldValue));
     }  else if (clazz == Double.class) {
-        doc.add(new DoubleField(field, (Double) fieldValue, Store.NO));
+        doc.add(new DoublePoint(field, (Double) fieldValue));
     } else {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c7a0d2a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
index 74f3742..e996f30 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
@@ -222,7 +222,7 @@ public class IndexRepositoryImplPerformanceTest {
         RegionDirectory dir = new RegionDirectory(new ConcurrentHashMap<String, File>(), new ConcurrentHashMap<ChunkKey, byte[]>());
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
         writer = new IndexWriter(dir, config);
-        searcherManager = new SearcherManager(writer, true, null);
+        searcherManager = new SearcherManager(writer, true, true, null);
       }
 
       @Override
@@ -275,7 +275,7 @@ public class IndexRepositoryImplPerformanceTest {
         RAMDirectory dir = new RAMDirectory();
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
         writer = new IndexWriter(dir, config);
-        searcherManager = new SearcherManager(writer, true, null);
+        searcherManager = new SearcherManager(writer, true, true, null);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c7a0d2a/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 18e327e..f89f281 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -71,7 +71,7 @@ jsr305.version = 3.0.1
 junit.version = 4.12
 JUnitParams.version = 1.0.4
 log4j.version = 2.5
-lucene.version = 5.3.0
+lucene.version = 6.0.0
 mockito-core.version = 1.10.19
 mockrunner.version = 1.0.8
 multithreadedtc.version = 1.01


[03/32] incubator-geode git commit: GEODE-1214: Better error handling for compilation failures

Posted by ji...@apache.org.
GEODE-1214: Better error handling for compilation failures


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 90ab09cc429643e3b7d9ec336fccbc6b2c0402c4
Parents: c67a1c9
Author: Jens Deppe <jd...@pivotal.io>
Authored: Tue Apr 12 09:01:00 2016 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Tue Apr 12 12:44:41 2016 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/internal/ClassBuilder.java    | 13 ++++++++++++-
 .../gemfire/internal/JarClassLoaderJUnitTest.java  | 17 +++++++++++++++++
 2 files changed, 29 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/90ab09cc/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassBuilder.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassBuilder.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassBuilder.java
index 118ad86..780ed4f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassBuilder.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/ClassBuilder.java
@@ -29,6 +29,8 @@ import java.util.List;
 import java.util.jar.JarEntry;
 import java.util.jar.JarOutputStream;
 
+import javax.tools.Diagnostic;
+import javax.tools.DiagnosticCollector;
 import javax.tools.FileObject;
 import javax.tools.ForwardingJavaFileManager;
 import javax.tools.JavaCompiler;
@@ -214,7 +216,16 @@ public class ClassBuilder implements Serializable {
     fileObjects.add(new JavaSourceFromString(className, classCode));
 
     List<String> options = Arrays.asList("-classpath", this.classPath);
-    javaCompiler.getTask(null, fileManager, null, options, null, fileObjects).call();
+    DiagnosticCollector<JavaFileObject> diagnostics = new DiagnosticCollector<JavaFileObject>();
+    if (! javaCompiler.getTask(null, fileManager, diagnostics, options, null, fileObjects).call()) {
+      StringBuilder errorMsg = new StringBuilder();
+      for (Diagnostic d : diagnostics.getDiagnostics()) {
+        String err = String.format("Compilation error: Line %d - %s%n", d.getLineNumber(), d.getMessage(null));
+        errorMsg.append(err);
+        System.err.print(err);
+      }
+      throw new IOException(errorMsg.toString());
+    }
     return byteArrayOutputStream.toByteArray();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/90ab09cc/geode-core/src/test/java/com/gemstone/gemfire/internal/JarClassLoaderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/JarClassLoaderJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/JarClassLoaderJUnitTest.java
index 20c7990..56b715f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/JarClassLoaderJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/JarClassLoaderJUnitTest.java
@@ -171,6 +171,23 @@ public class JarClassLoaderJUnitTest {
   }
 
   @Test
+  public void testFailingCompilation() throws Exception {
+    StringBuffer stringBuffer = new StringBuffer();
+    stringBuffer.append("import com.gemstone.gemfire.cache.Declarable;");
+    stringBuffer.append("import com.gemstone.gemfire.cache.execute.Function;");
+    stringBuffer.append("import com.gemstone.gemfire.cache.execute.FunctionContext;");
+    stringBuffer.append("public class JarClassLoaderJUnitFunction implements Function {}");
+    String functionString = stringBuffer.toString();
+
+    try {
+      this.classBuilder.createJarFromClassContent("JarClassLoaderJUnitFunction", functionString);
+      fail("This code should have failed to compile and thrown an exception");
+    } catch (Exception ex) {
+      // All good
+    }
+  }
+
+  @Test
   public void testFunctions() throws IOException, ClassNotFoundException {
     final File jarFile1 = new File(JAR_PREFIX + "JarClassLoaderJUnit.jar#1");
     final File jarFile2 = new File(JAR_PREFIX + "JarClassLoaderJUnit.jar#2");


[08/32] incubator-geode git commit: GEODE-1198 CI Failure: DistributedSystemDUnitTest.testConflictingUDPPort

Posted by ji...@apache.org.
GEODE-1198 CI Failure: DistributedSystemDUnitTest.testConflictingUDPPort

For this test, the membership-port-range has only 3 ports available. If in
some rare cases, one of the ports is used by others, it will get RMIException.
So just ignore it. Since adding one more port will also fail the test.


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 9d20f22ff99fc76bf2da3218f02ecb95cbf39b32
Parents: 80533ba
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Wed Apr 13 10:15:08 2016 -0700
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Wed Apr 13 10:19:08 2016 -0700

----------------------------------------------------------------------
 .../distributed/DistributedSystemDUnitTest.java | 20 +++++++++++++++++---
 1 file changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9d20f22f/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
index 0109845..22bb176 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
@@ -16,7 +16,10 @@
  */
 package com.gemstone.gemfire.distributed;
 
-import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.net.Inet4Address;
 import java.net.Inet6Address;
@@ -26,8 +29,12 @@ import java.util.Enumeration;
 import java.util.Properties;
 import java.util.concurrent.TimeoutException;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.GemFireConfigException;
+import com.gemstone.gemfire.SystemConnectException;
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
@@ -50,11 +57,10 @@ import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.LogWriterUtils;
+import com.gemstone.gemfire.test.dunit.RMIException;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
 
 /**
  * Tests the functionality of the {@link DistributedSystem} class.
@@ -387,6 +393,14 @@ public class DistributedSystemDUnitTest extends JUnit4DistributedTestCase {
           } catch (GemFireConfigException e) {
             return; // 
           }
+          catch (RMIException e) {
+            if (e.getCause() instanceof SystemConnectException) {
+              //GEODE-1198: for this test, the membership-port-range has only 3 ports available.
+              //If in some rare cases, one of the ports is used by others, it will get this 
+              //exception. So just ignore it. Since adding one more port will also fail the test.
+              return;
+            }
+          }
           fail("expected a GemFireConfigException but didn't get one");
         }
       });


[10/32] incubator-geode git commit: GEODE-1204 modified conditionals and opacity to make logo visible on Releases page.

Posted by ji...@apache.org.
GEODE-1204 modified conditionals and opacity to make logo visible on Releases page.

Closes #45971


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

Branch: refs/heads/feature/GEODE-17-2
Commit: f473facb33fea633782968513b792f13817c54a9
Parents: 0f692f8
Author: Dave Barnes <db...@pivotal.io>
Authored: Fri Apr 8 16:34:22 2016 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Wed Apr 13 10:57:48 2016 -0700

----------------------------------------------------------------------
 geode-site/website/Rules                 | 6 ++++++
 geode-site/website/layouts/default.html  | 2 +-
 geode-site/website/layouts/header.html   | 2 +-
 geode-site/website/layouts/releases.html | 1 +
 4 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f473facb/geode-site/website/Rules
----------------------------------------------------------------------
diff --git a/geode-site/website/Rules b/geode-site/website/Rules
index 5369385..46aead3 100644
--- a/geode-site/website/Rules
+++ b/geode-site/website/Rules
@@ -44,6 +44,12 @@ compile '/community/*' do
   layout 'community'
 end
 
+compile '/releases/*' do
+  @releases = true
+  filter :erb
+  layout 'releases'
+end
+
 compile '*' do
   if item.binary?
     # don’t filter binary items

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f473facb/geode-site/website/layouts/default.html
----------------------------------------------------------------------
diff --git a/geode-site/website/layouts/default.html b/geode-site/website/layouts/default.html
index 2e3e2cd..2bdf9ef 100644
--- a/geode-site/website/layouts/default.html
+++ b/geode-site/website/layouts/default.html
@@ -1,7 +1,7 @@
 <!DOCTYPE html>
 <html lang="en">
 
-<%= render 'header', {:docs => @docs, :community => @community} %>
+<%= render 'header', {:docs => @docs, :community => @community, :releases => @releases} %>
 
 <%= @content %>
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f473facb/geode-site/website/layouts/header.html
----------------------------------------------------------------------
diff --git a/geode-site/website/layouts/header.html b/geode-site/website/layouts/header.html
index 6edaec8..b5a4ca7 100644
--- a/geode-site/website/layouts/header.html
+++ b/geode-site/website/layouts/header.html
@@ -230,7 +230,7 @@
 </head>
 <body>
 
-    <header class="navbar navbar-inverse navbar-fixed-top bf-docs-nav <%= 'secondary' if @docs or @community %>" role="banner">
+    <header class="navbar navbar-inverse navbar-fixed-top bf-docs-nav <%= 'secondary' if @docs or @community or @releases %>" role="banner">
     <div class="container">
         <div class="navbar-header">
             <button class="navbar-toggle" type="button" data-toggle="collapse" data-target=".bf-navbar-collapse">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f473facb/geode-site/website/layouts/releases.html
----------------------------------------------------------------------
diff --git a/geode-site/website/layouts/releases.html b/geode-site/website/layouts/releases.html
new file mode 100644
index 0000000..9240770
--- /dev/null
+++ b/geode-site/website/layouts/releases.html
@@ -0,0 +1 @@
+<%= render 'default', :releases => true, :content => @content, :item => @item %>


[15/32] incubator-geode git commit: GEODE-1201: Reverting gradle changes

Posted by ji...@apache.org.
GEODE-1201: Reverting gradle changes


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 5c89fab63991da25a1262caf29a94bdc8eedbd6d
Parents: d1e4825
Author: Udo Kohlmeyer <uk...@pivotal.io>
Authored: Wed Apr 13 11:42:21 2016 +1000
Committer: Udo Kohlmeyer <uk...@pivotal.io>
Committed: Thu Apr 14 10:07:06 2016 +1000

----------------------------------------------------------------------
 geode-assembly/build.gradle | 6 ------
 1 file changed, 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5c89fab6/geode-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/geode-assembly/build.gradle b/geode-assembly/build.gradle
index 8f14fc1..b7d05e2 100755
--- a/geode-assembly/build.gradle
+++ b/geode-assembly/build.gradle
@@ -81,7 +81,6 @@ sourceSets {
 
 test {
   // test from the actual classpath not the gradle classpath
-  dependsOn copyRuntimeLibs
   dependsOn installDist
   // @TODO: this doesn't seem to be working need to get basename first.
   classpath += files "$buildDir/install/apache-geode/lib/geode-dependencies.jar"
@@ -113,11 +112,6 @@ task defaultCacheConfig(type: JavaExec, dependsOn: classes) {
   }
 }
 
-task copyRuntimeLibs(type: Copy) {
-  into "lib"
-  from configurations.testRuntime - configurations.runtime
-}
-
 // This closure sets the gemfire classpath.  If we add another jar to the classpath it must
 // be included in the filter logic below.
 def cp = {


[06/32] incubator-geode git commit: GEODE-1217: Marking jansi as optional in the pom

Posted by ji...@apache.org.
GEODE-1217: Marking jansi as optional in the pom


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

Branch: refs/heads/feature/GEODE-17-2
Commit: 69cd4a7e3f2d77d6d0790bf47ed82278b4d09d42
Parents: 6c033cf
Author: Dan Smith <up...@apache.org>
Authored: Tue Apr 12 13:38:20 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Wed Apr 13 10:08:42 2016 -0700

----------------------------------------------------------------------
 geode-core/build.gradle | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/69cd4a7e/geode-core/build.gradle
----------------------------------------------------------------------
diff --git a/geode-core/build.gradle b/geode-core/build.gradle
index 526f22a..8216d2a 100755
--- a/geode-core/build.gradle
+++ b/geode-core/build.gradle
@@ -88,7 +88,9 @@ dependencies {
 
   compile 'org.apache.logging.log4j:log4j-api:' + project.'log4j.version'
   compile 'org.apache.logging.log4j:log4j-core:' + project.'log4j.version'
-  runtime 'org.fusesource.jansi:jansi:' + project.'jansi.version'
+  runtime ('org.fusesource.jansi:jansi:' + project.'jansi.version') {
+    ext.optional = true
+  }
   runtime ('org.apache.logging.log4j:log4j-slf4j-impl:' + project.'log4j.version') {
     ext.optional = true
   }