You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2015/10/15 23:35:39 UTC

[16/38] incubator-geode git commit: GEODE-243: remove deprecated Bridge feature

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerTestCase.java
new file mode 100644
index 0000000..c645549
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerTestCase.java
@@ -0,0 +1,376 @@
+/*=========================================================================
+ * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
+ * This product is protected by U.S. and international copyright
+ * and intellectual property laws. Pivotal products are covered by
+ * one or more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.cache30;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Properties;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Declarable;
+import com.gemstone.gemfire.cache.LoaderHelper;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.client.Pool;
+import com.gemstone.gemfire.cache.client.PoolFactory;
+import com.gemstone.gemfire.cache.client.PoolManager;
+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.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+
+import dunit.VM;
+
+/**
+ * Provides helper methods for testing clients and servers. This
+ * test case was created by refactoring methods from ConnectionPoolDUnitTest into
+ * this class.
+ *
+ * @author Kirk Lund
+ * @since 4.2.1
+ */
+public class ClientServerTestCase extends CacheTestCase {
+  
+  public static String NON_EXISTENT_KEY = "NON_EXISTENT_KEY";
+  
+  public static boolean AUTO_LOAD_BALANCE = false;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    // this makes sure we don't have any connection left over from previous tests
+    disconnectAllFromDS();
+  }
+  
+  @Override
+  public void tearDown2() throws Exception {
+    // this makes sure we don't leave anything for the next tests
+    disconnectAllFromDS();
+  }
+
+  public ClientServerTestCase(String name) {
+    super(name);
+  }
+
+  /**
+   * Starts a bridge server on the given port
+   *
+   * @since 4.0
+   */
+  public int startBridgeServer(int port)
+    throws IOException {
+
+    Cache cache = getCache();
+    CacheServer bridge = cache.addCacheServer();
+    bridge.setPort(port);
+    bridge.setMaxThreads(getMaxThreads());
+    bridge.start();
+    return bridge.getPort();
+  }
+
+  /**
+   * Defaults to 0 which means no selector in server.
+   * Subclasses can override setting this to a value > 0 to enable selector.
+   */
+  protected int getMaxThreads() {
+    return 0;
+  }
+  
+  /**
+   * Stops the bridge server that serves up the given cache.
+   *
+   * @since 4.0
+   */
+  public void stopBridgeServers(Cache cache) {
+    CacheServer bridge = null;
+    for (Iterator bsI = cache.getCacheServers().iterator();bsI.hasNext(); ) {
+      bridge = (CacheServer) bsI.next();
+    bridge.stop();
+    assertFalse(bridge.isRunning());
+  }
+  }
+
+  /**
+   * Returns region attributes for a <code>LOCAL</code> region
+   */
+  protected RegionAttributes getRegionAttributes() {
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    return factory.create();
+  }
+
+  public static String createBridgeClientConnection(String host, int[] ports) {
+    StringBuffer sb = new StringBuffer();
+    for (int i = 0; i < ports.length; i++) {
+      if (i > 0) {
+        sb.append(",");
+      }
+      sb.append("name" + i + "=");
+      sb.append(host + ":" + ports[i]);
+    }
+    return sb.toString();
+  }
+
+  public static Pool configureConnectionPool(AttributesFactory factory,
+      String host, int port1, int port2, boolean establish, int redundancy,
+      int connectionsPerServer, String serverGroup, int pingInterval,
+      int idleTimeout, boolean threadLocalCnxs, int lifetimeTimeout) {
+    int[] ports;
+    if (port2 != -1) {
+      ports = new int[] { port1, port2 };
+    }
+    else {
+      ports = new int[] { port1 };
+    }
+    return configureConnectionPool(factory, host, ports, establish, redundancy,
+        connectionsPerServer, serverGroup, pingInterval, idleTimeout,
+        threadLocalCnxs, lifetimeTimeout);
+  }
+
+  public static Pool configureConnectionPool(AttributesFactory factory,
+      String host, int port1, int port2, boolean establish, int redundancy,
+      int connectionsPerServer, String serverGroup, int pingInterval,
+      int idleTimeout, boolean threadLocalCnxs) {
+    return configureConnectionPool(factory, host, port1, port2, establish,
+        redundancy, connectionsPerServer, serverGroup, pingInterval,
+        idleTimeout, threadLocalCnxs, -2/*lifetimeTimeout*/);
+  }
+
+  public static Pool configureConnectionPool(AttributesFactory factory,
+      String host, int port1, int port2, boolean establish, int redundancy,
+      int connectionsPerServer, String serverGroup, int pingInterval) {
+    return configureConnectionPool(factory, host, port1, port2, establish,
+        redundancy, connectionsPerServer, serverGroup, pingInterval, -1, false);
+  }
+
+  public static Pool configureConnectionPool(AttributesFactory factory,
+      String host, int port1, int port2, boolean establish, int redundancy,
+      int connectionsPerServer, String serverGroup) {
+    return configureConnectionPool(factory, host, port1, port2, establish,
+        redundancy, connectionsPerServer, serverGroup, -1/*pingInterval*/);
+  }
+
+  public static Pool configureConnectionPoolWithName(AttributesFactory factory,
+      String host, int[] ports, boolean establish, int redundancy,
+      int connectionsPerServer, String serverGroup, String poolName) {
+    return configureConnectionPoolWithNameAndFactory(factory, host, ports,
+        establish, redundancy, connectionsPerServer, serverGroup, poolName,
+        PoolManager.createFactory(), -1, -1, false, -2, -1);
+  }
+
+  public static Pool configureConnectionPoolWithName(AttributesFactory factory,
+      String host, int[] ports, boolean establish, int redundancy,
+      int connectionsPerServer, String serverGroup, String poolName,
+      int pingInterval, int idleTimeout, boolean threadLocalCnxs,
+      int lifetimeTimeout, int statisticInterval) {
+    return configureConnectionPoolWithNameAndFactory(factory, host, ports,
+        establish, redundancy, connectionsPerServer, serverGroup, poolName,
+        PoolManager.createFactory(), pingInterval, idleTimeout,
+        threadLocalCnxs, lifetimeTimeout, statisticInterval);
+  }
+
+  public static Pool configureConnectionPoolWithNameAndFactory(
+      AttributesFactory factory, String host, int[] ports, boolean establish,
+      int redundancy, int connectionsPerServer, String serverGroup,
+      String poolName, PoolFactory pf) {
+    return configureConnectionPoolWithNameAndFactory(factory, host, ports,
+        establish, redundancy, connectionsPerServer, serverGroup, poolName, pf,
+        -1, -1, false, -2, -1);
+  }
+
+  public static Pool configureConnectionPoolWithNameAndFactory(
+      AttributesFactory factory, String host, int[] ports, boolean establish,
+      int redundancy, int connectionsPerServer, String serverGroup,
+      String poolName, PoolFactory pf, int pingInterval, int idleTimeout,
+      boolean threadLocalCnxs, int lifetimeTimeout, int statisticInterval) {
+
+    if(AUTO_LOAD_BALANCE) {
+      pf.addLocator(host,getDUnitLocatorPort());
+    } else {
+      for(int z=0;z<ports.length;z++) {
+        pf.addServer(host,ports[z]);
+      }
+    }
+    
+    //TODO - probably should pass in minConnections rather than connecions per server
+    if(connectionsPerServer!=-1) {
+      pf.setMinConnections(connectionsPerServer * ports.length);
+    }
+    if (threadLocalCnxs) {
+      pf.setThreadLocalConnections(true);
+    }
+    if (pingInterval != -1) {
+      pf.setPingInterval(pingInterval);
+    }
+    if (idleTimeout != -1) {
+      pf.setIdleTimeout(idleTimeout);
+    }
+    if (statisticInterval != -1) {
+      pf.setStatisticInterval(statisticInterval);
+    }
+    if (lifetimeTimeout != -2) {
+      pf.setLoadConditioningInterval(lifetimeTimeout);
+    }
+    if(establish) {
+      pf.setSubscriptionEnabled(true);
+      pf.setSubscriptionRedundancy(redundancy);
+      pf.setSubscriptionAckInterval(1);
+    }
+    if(serverGroup!=null) {
+      pf.setServerGroup(serverGroup);
+    }
+    String rpoolName = "testPool";
+    if(poolName!=null) {
+      rpoolName = poolName;
+    }
+    Pool pool  = pf.create(rpoolName);
+    if(factory!=null) {
+      factory.setPoolName(rpoolName);
+    }
+    return pool;
+  }
+
+  public static Pool configureConnectionPool(AttributesFactory factory,
+      String host, int[] ports, boolean establish, int redundancy,
+      int connectionsPerServer, String serverGroup) {
+    return configureConnectionPool(factory, host, ports, establish, redundancy,
+        connectionsPerServer, serverGroup, -1/*pingInterval*/,
+        -1/*idleTimeout*/, false/*threadLocalCnxs*/, -2/*lifetimeTimeout*/);
+  }
+
+  public static Pool configureConnectionPool(AttributesFactory factory,
+      String host, int[] ports, boolean establish, int redundancy,
+      int connectionsPerServer, String serverGroup, int pingInterval,
+      int idleTimeout, boolean threadLocalCnxs, int lifetimeTimeout) {
+    return configureConnectionPoolWithName(factory, host, ports, establish,
+        redundancy, connectionsPerServer, serverGroup, null/*poolName*/,
+        pingInterval, idleTimeout, threadLocalCnxs, lifetimeTimeout, -1);
+  }
+
+  public static Pool configureConnectionPool(AttributesFactory factory,
+      String host, int[] ports, boolean establish, int redundancy,
+      int connectionsPerServer, String serverGroup, int pingInterval,
+      int idleTimeout, boolean threadLocalCnxs, int lifetimeTimeout,
+      int statisticInterval) {
+    return configureConnectionPoolWithName(factory, host, ports, establish,
+        redundancy, connectionsPerServer, serverGroup, null/*poolName*/,
+        pingInterval, idleTimeout, threadLocalCnxs, lifetimeTimeout,
+        statisticInterval);
+  }
+
+  /*protected static InternalDistributedMember findDistributedMember() {
+    DM dm = ((InternalDistributedSystem)
+      InternalDistributedSystem.getAnyInstance()).getDistributionManager();
+    return dm.getDistributionManagerId();
+  }*/
+
+  protected static String getMemberId() {
+    final InternalDistributedSystem system = InternalDistributedSystem.getAnyInstance();
+    WaitCriterion w = new WaitCriterion() {
+
+      public String description() {
+        return "bridge never finished connecting";
+      }
+
+      public boolean done() {
+//        getLogWriter().warning("checking member id " + system.getMemberId() +
+//            " for member " + system.getDistributedMember() + " hash " +
+//            System.identityHashCode(system.getDistributedMember()));
+        return !system.getMemberId().contains("):0:");
+      }
+      
+    };
+    int waitMillis = 5000;
+    int interval = 100;
+    boolean throwException = true;
+    waitForCriterion(w, waitMillis, interval, throwException);
+    return system.getMemberId();
+  }
+
+  protected static DistributedMember getDistributedMember() {
+    DistributedSystem system = InternalDistributedSystem.getAnyInstance();
+    return system.getDistributedMember();
+  }
+
+  protected static Properties getSystemProperties() {
+    DistributedSystem system = InternalDistributedSystem.getAnyInstance();
+    return system.getProperties();
+  }
+
+  public static class CacheServerCacheLoader extends TestCacheLoader implements Declarable {
+
+    public CacheServerCacheLoader() {}
+
+    @Override
+    public Object load2(LoaderHelper helper) {
+      if (helper.getArgument() instanceof Integer) {
+        try {
+          Thread.sleep(((Integer) helper.getArgument()).intValue());
+        }
+        catch (InterruptedException ugh) { fail("interrupted"); }
+      }
+      Object ret = helper.getKey();
+      
+      if( ret instanceof String)
+      {
+        if(ret != null && ret.equals(NON_EXISTENT_KEY))
+          return null;//return null
+      }
+      return ret;
+      
+    }
+
+    public void init(Properties props)  {}
+  }
+
+  public final static String BridgeServerKey = "BridgeServerKey";
+  /**
+   * Create a server that has a value for every key queried and a unique
+   * key/value in the specified Region that uniquely identifies each instance.
+   *
+   * @param vm
+   *          the VM on which to create the server
+   * @param rName
+   *          the name of the Region to create on the server
+   * @param port
+   *          the TCP port on which the server should listen
+   */
+  public void createBridgeServer(VM vm, final String rName, final int port) {
+    vm.invoke(new CacheSerializableRunnable("Create Region on Server") {
+    @Override
+    public void run2() {
+      try {
+        AttributesFactory factory = new AttributesFactory();
+        factory.setScope(Scope.DISTRIBUTED_ACK); // can't be local since used with registerInterest
+        factory.setCacheLoader(new CacheServerCacheLoader());
+        beginCacheXml();
+        createRootRegion(rName, factory.create());
+        startBridgeServer(port);
+        finishCacheXml(rName + "-" + port);
+
+        Region region = getRootRegion(rName);
+        assertNotNull(region);
+        region.put(BridgeServerKey, new Integer(port)); // A unique key/value to identify the BridgeServer
+      }
+      catch(Exception e) {
+        getSystem().getLogWriter().severe(e);
+        fail("Failed to start CacheServer " + e);
+      }
+    }
+  });
+  }
+
+  public static int[] createUniquePorts(int numToCreate) {
+    return AvailablePortHelper.getRandomAvailableTCPPorts(numToCreate);
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractServerLauncherJUnitTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractServerLauncherJUnitTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractServerLauncherJUnitTestCase.java
index dc506ad..ca676a5 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractServerLauncherJUnitTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractServerLauncherJUnitTestCase.java
@@ -14,7 +14,7 @@ import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
 import com.gemstone.gemfire.distributed.ServerLauncher.ServerState;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.internal.cache.AbstractBridgeServer;
+import com.gemstone.gemfire.internal.cache.AbstractCacheServer;
 
 /**
  * @author Kirk Lund
@@ -35,7 +35,7 @@ public abstract class AbstractServerLauncherJUnitTestCase extends AbstractLaunch
   public final void setUpServerLauncherTest() throws Exception {
     System.setProperty("gemfire." + DistributionConfig.MCAST_PORT_NAME, Integer.toString(0));
     final int port = AvailablePortHelper.getRandomAvailableTCPPort();
-    System.setProperty(AbstractBridgeServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, String.valueOf(port));
+    System.setProperty(AbstractCacheServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, String.valueOf(port));
     this.serverPort = port;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
index 44849f8..e83c8ac 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
@@ -21,7 +21,7 @@ import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.GemFireVersion;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.cache.AbstractBridgeServer;
+import com.gemstone.gemfire.internal.cache.AbstractCacheServer;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlGenerator;
 import com.gemstone.gemfire.internal.cache.xmlcache.RegionAttributesCreation;
@@ -605,7 +605,7 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
       this.launcher.start();
      
       // why did it not fail like it's supposed to?
-      final String property = System.getProperty(AbstractBridgeServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY);
+      final String property = System.getProperty(AbstractCacheServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY);
       assertNotNull(property);
       assertEquals(this.serverPort, Integer.valueOf(property).intValue());
       assertFalse(AvailablePort.isPortAvailable(this.serverPort, AvailablePort.SOCKET));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
index 1ba1189..d43ad0a 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
@@ -40,7 +40,7 @@ import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.GemFireVersion;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.cache.AbstractBridgeServer;
+import com.gemstone.gemfire.internal.cache.AbstractCacheServer;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlGenerator;
 import com.gemstone.gemfire.internal.cache.xmlcache.RegionAttributesCreation;
@@ -397,7 +397,7 @@ public class ServerLauncherRemoteJUnitTest extends AbstractServerLauncherJUnitTe
     
     // build and start the server
     final List<String> jvmArguments = getJvmArguments();
-    jvmArguments.add("-D" + AbstractBridgeServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY + "=" + this.serverPort);
+    jvmArguments.add("-D" + AbstractCacheServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY + "=" + this.serverPort);
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -463,7 +463,7 @@ public class ServerLauncherRemoteJUnitTest extends AbstractServerLauncherJUnitTe
     
     // build and start the server
     final List<String> jvmArguments = getJvmArguments();
-    jvmArguments.add("-D" + AbstractBridgeServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY + "=" + this.serverPort);
+    jvmArguments.add("-D" + AbstractCacheServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY + "=" + this.serverPort);
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -832,7 +832,7 @@ public class ServerLauncherRemoteJUnitTest extends AbstractServerLauncherJUnitTe
     
     // launch server
     final List<String> jvmArguments = getJvmArguments();
-    jvmArguments.add("-D" + AbstractBridgeServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY + "=" + this.serverPort);
+    jvmArguments.add("-D" + AbstractCacheServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY + "=" + this.serverPort);
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogDUnitTest.java
index 6a20f15..d4eab5d 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogDUnitTest.java
@@ -18,7 +18,7 @@ import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 
 import dunit.DistributedTestCase;
 import dunit.Host;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
index 79d6f64..447b32e 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
@@ -25,7 +25,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ReplyException;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.BucketRegion;
 import com.gemstone.gemfire.internal.cache.DistTXState;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -130,7 +130,7 @@ public class DistributedTransactionDUnitTest extends CacheTestCase {
         int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
         CacheServer s = getCache().addCacheServer();
         s.setPort(port);
-        ((BridgeServerImpl) s).setTransactionTimeToLive(10);
+        ((CacheServerImpl) s).setTransactionTimeToLive(10);
         s.start();
         return port;
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java
index ca1af49..6f52a0d 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug39079DUnitTest.java
@@ -27,7 +27,7 @@ import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.distributed.DistributedSystem;
@@ -322,7 +322,7 @@ public class Bug39079DUnitTest extends CacheTestCase {
     
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(gemfirecache, props, Scope.DISTRIBUTED_ACK);
     assertNotNull(region);
-    BridgeServer bs1 = gemfirecache.addBridgeServer();
+    CacheServer bs1 = gemfirecache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     bs1.setPort(port);
     bs1.start();
@@ -369,7 +369,7 @@ public class Bug39079DUnitTest extends CacheTestCase {
       assertTrue(region.getRegionService().isClosed());
       
       region = null;
-      List bsRunning = gemfirecache.getBridgeServers();
+      List bsRunning = gemfirecache.getCacheServers();
       assertTrue(bsRunning.isEmpty());
     }
     finally {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug41957DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug41957DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug41957DUnitTest.java
index 67b6790..81fdadb 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug41957DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug41957DUnitTest.java
@@ -12,7 +12,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 
 import com.gemstone.gemfire.cache.client.*;
@@ -30,7 +30,7 @@ import java.util.*;
  * @author darrel
  * @since 6.5
  */
- public class Bug41957DUnitTest extends BridgeTestCase {
+ public class Bug41957DUnitTest extends ClientServerTestCase {
 
   public Bug41957DUnitTest(String name) {
     super(name);
@@ -79,7 +79,7 @@ import java.util.*;
 
         // Create Region
         AttributesFactory factory = new AttributesFactory();
-        factory.setCacheLoader(new BridgeServerCacheLoader());
+        factory.setCacheLoader(new CacheServerCacheLoader());
         if (createPR) {
           factory.setDataPolicy(DataPolicy.PARTITION);
           factory.setPartitionAttributes((new PartitionAttributesFactory()).create());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientMessagesRegionCreationAndDestroyJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientMessagesRegionCreationAndDestroyJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientMessagesRegionCreationAndDestroyJUnitTest.java
index 21c1a5b..196f3ee 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientMessagesRegionCreationAndDestroyJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientMessagesRegionCreationAndDestroyJUnitTest.java
@@ -67,7 +67,7 @@ public class ClientMessagesRegionCreationAndDestroyJUnitTest {
    */
   
   private void attachBridgeServer() throws IOException {
-    BridgeServerImpl server = (BridgeServerImpl)cache.addBridgeServer();
+    CacheServerImpl server = (CacheServerImpl)cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -80,7 +80,7 @@ public class ClientMessagesRegionCreationAndDestroyJUnitTest {
         Region.SEPARATOR + regionName).getAttributes().getEvictionAttributes();
     assertTrue("Eviction Algorithm is not LIFO", ea.isLIFO());
     // The CacheClientNotifier is a singleton. 
-    if (cache.getBridgeServers().size() <= 1) {
+    if (cache.getCacheServers().size() <= 1) {
       assertTrue("client messages region name should not be present ", (regionNames).add(regionName));
     } else {
       assertTrue("client messages region name should have been already present ", (regionNames).contains(regionName));      
@@ -106,7 +106,7 @@ public class ClientMessagesRegionCreationAndDestroyJUnitTest {
    * Attach bridge server
    */
   private void attachmentOfBridgeServer() {
-    if (cache.getBridgeServers().size() < brigeNum) {
+    if (cache.getCacheServers().size() < brigeNum) {
       try {
         // attaching and starting bridge server
         attachBridgeServer();
@@ -122,8 +122,8 @@ public class ClientMessagesRegionCreationAndDestroyJUnitTest {
    */
   private void dettachmentOfBridgeServer() {
     // detach all bridge server to test destroy of client_messages_region
-    for (Iterator itr = cache.getBridgeServers().iterator(); itr.hasNext();) {
-      BridgeServerImpl server = (BridgeServerImpl)itr.next();
+    for (Iterator itr = cache.getCacheServers().iterator(); itr.hasNext();) {
+      CacheServerImpl server = (CacheServerImpl)itr.next();
       String rName = ((HAContainerWrapper)server.getAcceptor().getCacheClientNotifier().getHaContainer()).getName();
       assertNotNull("client messages region is null ", cache.getRegion(Region.SEPARATOR + rName));
       server.stop();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerGetAllDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerGetAllDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerGetAllDUnitTest.java
index 5a9cd77..8716ccb 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerGetAllDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerGetAllDUnitTest.java
@@ -13,10 +13,10 @@ import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.cache.client.*;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 
 import dunit.*;
 
@@ -28,7 +28,7 @@ import java.util.*;
  * @author Barry Oglesby
  * @since 5.7
  */
- public class ClientServerGetAllDUnitTest extends BridgeTestCase {
+ public class ClientServerGetAllDUnitTest extends ClientServerTestCase {
 
   public ClientServerGetAllDUnitTest(String name) {
     super(name);
@@ -63,7 +63,7 @@ import java.util.*;
           keys.add("key-"+i);
         }
         
-        keys.add(BridgeTestCase.NON_EXISTENT_KEY); // this will not be load CacheLoader
+        keys.add(ClientServerTestCase.NON_EXISTENT_KEY); // this will not be load CacheLoader
         
         // Invoke getAll
         Region region = getRootRegion(regionName);
@@ -79,13 +79,13 @@ import java.util.*;
           String key = (String) i.next();
           assertTrue(result.containsKey(key));
           Object value = result.get(key);
-          if(!key.equals(BridgeTestCase.NON_EXISTENT_KEY))
+          if(!key.equals(ClientServerTestCase.NON_EXISTENT_KEY))
             assertEquals(key, value);
           else
             assertEquals(null, value);
         }
         
-        assertEquals(null, region.get(BridgeTestCase.NON_EXISTENT_KEY));
+        assertEquals(null, region.get(ClientServerTestCase.NON_EXISTENT_KEY));
       }
     });
 
@@ -115,7 +115,7 @@ import java.util.*;
           keys.add("key-"+i);
         }
         
-        keys.add(BridgeTestCase.NON_EXISTENT_KEY); // this will not be load CacheLoader
+        keys.add(ClientServerTestCase.NON_EXISTENT_KEY); // this will not be load CacheLoader
         
         // Invoke getAll
         Region region = getRootRegion(regionName);
@@ -131,13 +131,13 @@ import java.util.*;
           String key = (String) i.next();
           assertTrue(result.containsKey(key));
           Object value = result.get(key);
-          if(!key.equals(BridgeTestCase.NON_EXISTENT_KEY))
+          if(!key.equals(ClientServerTestCase.NON_EXISTENT_KEY))
             assertEquals(key, value);
           else
             assertEquals(null, value);
         }
         
-        assertEquals(null, region.get(BridgeTestCase.NON_EXISTENT_KEY));
+        assertEquals(null, region.get(ClientServerTestCase.NON_EXISTENT_KEY));
       }
     });
     checkServerForOrphans(server, regionName);
@@ -392,7 +392,7 @@ import java.util.*;
           keys.add("key-"+i);
         }
         
-        keys.add(BridgeTestCase.NON_EXISTENT_KEY); // this will not be load CacheLoader
+        keys.add(ClientServerTestCase.NON_EXISTENT_KEY); // this will not be load CacheLoader
         
         // Invoke getAll
         Region region = getRootRegion(regionName);
@@ -408,13 +408,13 @@ import java.util.*;
           String key = (String) i.next();
           assertTrue(result.containsKey(key));
           Object value = result.get(key);
-          if(!key.equals(BridgeTestCase.NON_EXISTENT_KEY))
+          if(!key.equals(ClientServerTestCase.NON_EXISTENT_KEY))
             assertEquals(key, value);
           else
             assertEquals(null, value);
         }
         
-        assertEquals(null, region.get(BridgeTestCase.NON_EXISTENT_KEY));
+        assertEquals(null, region.get(ClientServerTestCase.NON_EXISTENT_KEY));
       }
     });
 
@@ -467,7 +467,7 @@ import java.util.*;
         for (int i=0; i<5; i++) {
           keys.add("key-"+i);
         }
-        keys.add(BridgeTestCase.NON_EXISTENT_KEY); // this will not be load CacheLoader
+        keys.add(ClientServerTestCase.NON_EXISTENT_KEY); // this will not be load CacheLoader
         
         // Invoke getAll
         
@@ -484,12 +484,12 @@ import java.util.*;
           String key = (String) i.next();
           assertTrue(result.containsKey(key));
           Object value = result.get(key);
-          if(!key.equals(BridgeTestCase.NON_EXISTENT_KEY))
+          if(!key.equals(ClientServerTestCase.NON_EXISTENT_KEY))
             assertEquals(key, value);
           else
             assertEquals(null, value);
         }
-        assertEquals(null, region.get(BridgeTestCase.NON_EXISTENT_KEY));
+        assertEquals(null, region.get(ClientServerTestCase.NON_EXISTENT_KEY));
       }
     });
 
@@ -663,9 +663,9 @@ import java.util.*;
           factory.setOffHeap(true);
         }
         if (expectCallback) {
-          factory.setCacheLoader(new CallbackBridgeServerCacheLoader());
+          factory.setCacheLoader(new CallbackCacheServerCacheLoader());
         } else {
-          factory.setCacheLoader(new BridgeServerCacheLoader());
+          factory.setCacheLoader(new CacheServerCacheLoader());
         }
         if (createPR) {
           factory.setDataPolicy(DataPolicy.PARTITION);
@@ -680,7 +680,7 @@ import java.util.*;
         }
         try {
           Cache cache = getCache();
-          BridgeServer bridge = cache.addBridgeServer();
+          CacheServer bridge = cache.addCacheServer();
           bridge.setPort(serverPort);
           // for off-heap I want the server to use a selector
           bridge.setMaxThreads(offheap ? 16 : getMaxThreads());
@@ -694,7 +694,7 @@ import java.util.*;
   
   private static final String CALLBACK_ARG = "ClientServerGetAllDUnitTestCB";
 
-  private static class CallbackBridgeServerCacheLoader extends BridgeServerCacheLoader {
+  private static class CallbackCacheServerCacheLoader extends CacheServerCacheLoader {
     @Override
     public Object load2(LoaderHelper helper) {
       if (helper.getArgument() instanceof String) {
@@ -702,7 +702,7 @@ import java.util.*;
           fail("Expected " + helper.getArgument() + " to be " + CALLBACK_ARG);
         }
       } else {
-        if (!helper.getKey().equals(BridgeTestCase.NON_EXISTENT_KEY)) {
+        if (!helper.getKey().equals(ClientServerTestCase.NON_EXISTENT_KEY)) {
           fail("Expected callback arg to be " + CALLBACK_ARG + " but it was null");
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
index 4b65a95..1bf0953 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
@@ -100,7 +100,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
           int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
           CacheServer s = getCache().addCacheServer();
           s.setPort(port);
-          ((BridgeServerImpl)s).setTransactionTimeToLive(10);
+          ((CacheServerImpl)s).setTransactionTimeToLive(10);
           s.start();
           return port;
         }
@@ -125,7 +125,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
           int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
           CacheServer s = cache.addCacheServer();
           s.setPort(port);
-          ((BridgeServerImpl)s).setTransactionTimeToLive(10);
+          ((CacheServerImpl)s).setTransactionTimeToLive(10);
           s.start();
           return port;
         }
@@ -193,7 +193,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
         int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
         CacheServer s = getCache().addCacheServer();
         s.setPort(port);
-        ((BridgeServerImpl)s).setTransactionTimeToLive(10);
+        ((CacheServerImpl)s).setTransactionTimeToLive(10);
         s.start();
         return port;
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentMapOpsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentMapOpsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentMapOpsDUnitTest.java
index a34f6d2..88ce70a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentMapOpsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentMapOpsDUnitTest.java
@@ -48,7 +48,7 @@ import dunit.SerializableRunnable;
 import dunit.VM;
 
 /**
- * tests for the concurrentMapOperations. there are more tests in BridgeWriterMiscDUnitTest
+ * tests for the concurrentMapOperations. there are more tests in ClientServerMiscDUnitTest
  * @author sbawaska
  *
  */
@@ -1067,7 +1067,7 @@ public class ConcurrentMapOpsDUnitTest extends CacheTestCase {
             r.destroy("key0");
           }
           // force client to use server1 for now
-//          getCache().getBridgeServers().get(0).stop();
+//          getCache().getCacheServers().get(0).stop();
           r.getAttributesMutator().addCacheListener(new CacheListenerAdapter() {
             private void killSender(EntryEvent event) {
               if (event.isOriginRemote()) {
@@ -1104,7 +1104,7 @@ public class ConcurrentMapOpsDUnitTest extends CacheTestCase {
         public Object call() throws Exception {
           Region r = getCache().getRegion(regionName);
           // force client to use server1 for now
-//          getCache().getBridgeServers().get(0).stop();
+//          getCache().getCacheServers().get(0).stop();
           r.getAttributesMutator().addCacheListener(new CacheListenerAdapter() {
             private void killSender(EntryEvent event) {
               if (event.isOriginRemote()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationDUnitTest.java
index f599687..c7d7923 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationDUnitTest.java
@@ -37,7 +37,7 @@ import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.compression.SnappyCompressor;
 import com.gemstone.gemfire.distributed.DistributedSystem;
@@ -877,7 +877,7 @@ public class DeltaPropagationDUnitTest extends DistributedTestCase {
   public static void confirmEviction(Integer port) {
     final EnableLRU cc = ((VMLRURegionMap)((LocalRegion)cache
         .getRegion(Region.SEPARATOR
-            + BridgeServerImpl.generateNameForClientMsgsRegion(port))).entries)
+            + CacheServerImpl.generateNameForClientMsgsRegion(port))).entries)
         ._getCCHelper();
 
     WaitCriterion wc = new WaitCriterion() {
@@ -1376,7 +1376,7 @@ public class DeltaPropagationDUnitTest extends DistributedTestCase {
     props.setProperty(DistributionConfig.CLIENT_CONFLATION_PROP_NAME, conflate);
     new DeltaPropagationDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
-    pool = BridgeTestCase.configureConnectionPool(factory, "localhost", ports,
+    pool = ClientServerTestCase.configureConnectionPool(factory, "localhost", ports,
         true, Integer.parseInt(rLevel), 2, null, 1000, 250, false, -2);
 
     factory.setScope(Scope.LOCAL);
@@ -1423,7 +1423,7 @@ public class DeltaPropagationDUnitTest extends DistributedTestCase {
     try {
       // Get the clientMessagesRegion and check the size.
       Region region = (Region)cache.getRegion("/" + regionName);
-      Region msgsRegion = (Region)cache.getRegion(BridgeServerImpl
+      Region msgsRegion = (Region)cache.getRegion(CacheServerImpl
           .generateNameForClientMsgsRegion(port.intValue()));
       logger.fine("size<serverRegion, clientMsgsRegion>: " + region.size()
           + ", " + msgsRegion.size());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationStatsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationStatsDUnitTest.java
index e060b0d..968e3e8 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationStatsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DeltaPropagationStatsDUnitTest.java
@@ -463,7 +463,7 @@ public class DeltaPropagationStatsDUnitTest extends DistributedTestCase {
       numOfDeltasSent = region.getCachePerfStats().getDeltasSent();
       deltaTime = region.getCachePerfStats().getDeltasPreparedTime();
     } else if (path == SERVER_TO_CLIENT) {
-      CacheClientNotifier ccn = ((BridgeServerImpl)cache.getCacheServers()
+      CacheClientNotifier ccn = ((CacheServerImpl)cache.getCacheServers()
           .toArray()[0]).getAcceptor().getCacheClientNotifier();
 
       numOfDeltasSent = ((CacheClientProxy)ccn.getClientProxies().toArray()[0])

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java
index 9f85b6f..0c7328b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/DiskRegionJUnitTest.java
@@ -24,7 +24,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import static org.junit.Assert.*;
-
 import junit.framework.Assert;
 
 import com.gemstone.gemfire.SystemFailure;
@@ -34,7 +33,7 @@ import com.gemstone.gemfire.cache.EntryEvent;
 import com.gemstone.gemfire.cache.EntryNotFoundException;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.internal.cache.lru.LRUStatistics;
 import com.gemstone.gemfire.internal.cache.lru.NewLRUClockHand;
@@ -2341,7 +2340,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       props.setPersistBackup(true);
   
       region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, props, Scope.LOCAL);
-      BridgeServer bs1 = cache.addBridgeServer();
+      CacheServer bs1 = cache.addCacheServer();
       bs1.setPort(5555);
       bs1.start();
 
@@ -2459,7 +2458,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       props.setMaxOplogSize(100000); // just needs to be bigger than 65550
 
       region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, props, Scope.LOCAL);
-      BridgeServer bs1 = cache.addBridgeServer();
+      CacheServer bs1 = cache.addCacheServer();
       bs1.setPort(5555);
       bs1.start();      
 
@@ -2488,7 +2487,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase
       }
       assertTrue(region.isDestroyed());        
       region = null;
-      List bsRunning = cache.getBridgeServers();
+      List bsRunning = cache.getCacheServers();
       assertTrue(!bsRunning.isEmpty());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/EventTrackerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/EventTrackerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/EventTrackerDUnitTest.java
index 9f1bc68..d72a9d6 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/EventTrackerDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/EventTrackerDUnitTest.java
@@ -23,7 +23,7 @@ import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -160,7 +160,7 @@ public class EventTrackerDUnitTest extends CacheTestCase {
         getCache();
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, hostName, port, -1, false, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, hostName, port, -1, false, -1, -1, null);
         createRegion(regionName, factory.create());
       }
     });
@@ -233,7 +233,7 @@ public class EventTrackerDUnitTest extends CacheTestCase {
         getCache();
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, hostName, port, -1, false, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, hostName, port, -1, false, -1, -1, null);
         createRegion(regionName, factory.create());
       }
     });
@@ -311,7 +311,7 @@ public class EventTrackerDUnitTest extends CacheTestCase {
         getCache();
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, hostName, port, -1, false, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, hostName, port, -1, false, -1, -1, null);
         createRootRegion("partitioned", factory.create());
         createRootRegion("replicate", factory.create());
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/GridAdvisorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/GridAdvisorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/GridAdvisorDUnitTest.java
index 33f3b6d..693fe48 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/GridAdvisorDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/GridAdvisorDUnitTest.java
@@ -8,7 +8,7 @@
 package com.gemstone.gemfire.internal.cache;
 
 import com.gemstone.gemfire.cache.*;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.*;
 import com.gemstone.gemfire.distributed.internal.*;
 import com.gemstone.gemfire.internal.AvailablePort.Keeper;
@@ -114,7 +114,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
         public void run() {
           try {
             Cache c = CacheFactory.getAnyInstance();
-            BridgeServer bs = c.addBridgeServer();
+            CacheServer bs = c.addCacheServer();
             bs.setPort(bsPort1);
             bs.setGroups(new String[] {"bs1Group1", "bs1Group2"});
             bs.start();
@@ -130,7 +130,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
           public void run() {
             try {
               Cache c = CacheFactory.getAnyInstance();
-              BridgeServer bs = c.addBridgeServer();
+              CacheServer bs = c.addCacheServer();
               bs.setPort(bsPort3);
               bs.setGroups(new String[] {"bs3Group1", "bs3Group2"});
               bs.start();
@@ -151,7 +151,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
       public void run() {
         try {
           Cache c = CacheFactory.getAnyInstance();
-          BridgeServer bs = c.addBridgeServer();
+          CacheServer bs = c.addCacheServer();
           bs.setPort(bsPort2);
           bs.setGroups(new String[] {"bs2Group1", "bs2Group2"});
           bs.start();
@@ -167,7 +167,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
       public void run() {
         try {
           Cache c = CacheFactory.getAnyInstance();
-          BridgeServer bs = c.addBridgeServer();
+          CacheServer bs = c.addCacheServer();
           bs.setPort(bsPort4);
           bs.setGroups(new String[] {"bs4Group1", "bs4Group2"});
           bs.start();
@@ -198,8 +198,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             others = ca.fetchBridgeServers();
             assertEquals(4, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort1) {
                 assertEquals(Arrays.asList(new String[] {"bs1Group1", "bs1Group2"}),
                              Arrays.asList(bsp.getGroups()));
@@ -234,8 +234,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             others = ca.fetchBridgeServers();
             assertEquals(4, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort1) {
                 assertEquals(Arrays.asList(new String[] {"bs1Group1", "bs1Group2"}),
                              Arrays.asList(bsp.getGroups()));
@@ -257,11 +257,11 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
     vm1.invoke(new SerializableRunnable("Verify bridge server view on " + bsPort1 + " and on " + bsPort3) {
         public void run() {
           Cache c = CacheFactory.getAnyInstance();
-          List bslist = c.getBridgeServers();
+          List bslist = c.getCacheServers();
           assertEquals(2, bslist.size());
           for (int i=0; i < bslist.size(); i++) {
             DistributionAdvisee advisee = (DistributionAdvisee)bslist.get(i);
-            BridgeServerAdvisor bsa = (BridgeServerAdvisor)advisee.getDistributionAdvisor();
+            CacheServerAdvisor bsa = (CacheServerAdvisor)advisee.getDistributionAdvisor();
             List others = bsa.fetchBridgeServers();
             getLogWriter().info("found these bridgeservers in " + advisee + ": " + others);
             assertEquals(3, others.size());
@@ -285,11 +285,11 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
     vm2.invoke(new SerializableRunnable("Verify bridge server view on " + bsPort2 + " and on " + bsPort4) {
         public void run() {
           Cache c = CacheFactory.getAnyInstance();
-          List bslist = c.getBridgeServers();
+          List bslist = c.getCacheServers();
           assertEquals(2, bslist.size());
           for (int i=0; i < bslist.size(); i++) {
             DistributionAdvisee advisee = (DistributionAdvisee)bslist.get(i);
-            BridgeServerAdvisor bsa = (BridgeServerAdvisor)advisee.getDistributionAdvisor();
+            CacheServerAdvisor bsa = (CacheServerAdvisor)advisee.getDistributionAdvisor();
             List others = bsa.fetchBridgeServers();
             getLogWriter().info("found these bridgeservers in " + advisee + ": " + others);
             assertEquals(3, others.size());
@@ -315,9 +315,9 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
       new SerializableRunnable("stop bridge server") {
           public void run() {
             Cache c = CacheFactory.getAnyInstance();
-            List bslist = c.getBridgeServers();
+            List bslist = c.getCacheServers();
             assertEquals(2, bslist.size());
-            BridgeServer bs = (BridgeServer)bslist.get(0);
+            CacheServer bs = (CacheServer)bslist.get(0);
             bs.stop();
           }
         };
@@ -342,8 +342,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             others = ca.fetchBridgeServers();
             assertEquals(3, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort2) {
                 assertEquals(Arrays.asList(new String[] {"bs2Group1", "bs2Group2"}),
                              Arrays.asList(bsp.getGroups()));
@@ -375,8 +375,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             others = ca.fetchBridgeServers();
             assertEquals(3, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort2) {
                 assertEquals(Arrays.asList(new String[] {"bs2Group1", "bs2Group2"}),
                              Arrays.asList(bsp.getGroups()));
@@ -427,11 +427,11 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
     vm2.invoke(new SerializableRunnable("Verify bridge server saw locator stop") {
         public void run() {
           Cache c = CacheFactory.getAnyInstance();
-          List bslist = c.getBridgeServers();
+          List bslist = c.getCacheServers();
           assertEquals(2, bslist.size());
           for (int i=0; i < bslist.size(); i++) {
             DistributionAdvisee advisee = (DistributionAdvisee)bslist.get(i);
-            BridgeServerAdvisor bsa = (BridgeServerAdvisor)advisee.getDistributionAdvisor();
+            CacheServerAdvisor bsa = (CacheServerAdvisor)advisee.getDistributionAdvisor();
             List others = bsa.fetchControllers();
             assertEquals(1, others.size());
             for (int j=0; j < others.size(); j++) {
@@ -450,7 +450,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
     vm1.invoke(new SerializableRunnable("Verify bridge server saw locator stop") {
         public void run() {
           Cache c = CacheFactory.getAnyInstance();
-          List bslist = c.getBridgeServers();
+          List bslist = c.getCacheServers();
           assertEquals(2, bslist.size());
           for (int i=0; i < bslist.size(); i++) {
             DistributionAdvisee advisee = (DistributionAdvisee)bslist.get(i);
@@ -458,7 +458,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
               // skip this one since it is stopped
               continue;
             }
-            BridgeServerAdvisor bsa = (BridgeServerAdvisor)advisee.getDistributionAdvisor();
+            CacheServerAdvisor bsa = (CacheServerAdvisor)advisee.getDistributionAdvisor();
             List others = bsa.fetchControllers();
             assertEquals(1, others.size());
             for (int j=0; j < others.size(); j++) {
@@ -480,9 +480,9 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
           public void run() {
             try {
               Cache c = CacheFactory.getAnyInstance();
-              List bslist = c.getBridgeServers();
+              List bslist = c.getCacheServers();
               assertEquals(2, bslist.size());
-              BridgeServer bs = (BridgeServer)bslist.get(0);
+              CacheServer bs = (CacheServer)bslist.get(0);
               bs.setHostnameForClients("nameForClients");
               bs.start();
             } catch (IOException ex) {
@@ -505,8 +505,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             List others = ca.fetchBridgeServers();
             assertEquals(4, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort1) {
                 assertEquals(Arrays.asList(new String[] {"bs1Group1", "bs1Group2"}),
                              Arrays.asList(bsp.getGroups()));
@@ -632,7 +632,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
         public void run() {
           try {
             Cache c = CacheFactory.getAnyInstance();
-            BridgeServer bs = c.addBridgeServer();
+            CacheServer bs = c.addCacheServer();
             bs.setPort(bsPort1);
             bs.start();
           } catch (IOException ex) {
@@ -647,7 +647,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
           public void run() {
             try {
               Cache c = CacheFactory.getAnyInstance();
-              BridgeServer bs = c.addBridgeServer();
+              CacheServer bs = c.addCacheServer();
               bs.setPort(bsPort3);
               bs.start();
             } catch (IOException ex) {
@@ -667,7 +667,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
       public void run() {
         try {
           Cache c = CacheFactory.getAnyInstance();
-          BridgeServer bs = c.addBridgeServer();
+          CacheServer bs = c.addCacheServer();
           bs.setPort(bsPort2);
           bs.start();
         } catch (IOException ex) {
@@ -682,7 +682,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
       public void run() {
         try {
           Cache c = CacheFactory.getAnyInstance();
-          BridgeServer bs = c.addBridgeServer();
+          CacheServer bs = c.addCacheServer();
           bs.setPort(bsPort4);
           bs.start();
         } catch (IOException ex) {
@@ -712,8 +712,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             others = ca.fetchBridgeServers();
             assertEquals(4, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort1) {
                 assertEquals(Arrays.asList(new String[] {"bs1Group1", "bs1Group2"}),
                              Arrays.asList(bsp.getGroups()));
@@ -748,8 +748,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             others = ca.fetchBridgeServers();
             assertEquals(4, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort1) {
                 assertEquals(Arrays.asList(new String[] {"bs1Group1", "bs1Group2"}),
                              Arrays.asList(bsp.getGroups()));
@@ -771,11 +771,11 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
     vm1.invoke(new SerializableRunnable("Verify bridge server view on " + bsPort1 + " and on " + bsPort3) {
         public void run() {
           Cache c = CacheFactory.getAnyInstance();
-          List bslist = c.getBridgeServers();
+          List bslist = c.getCacheServers();
           assertEquals(2, bslist.size());
           for (int i=0; i < bslist.size(); i++) {
             DistributionAdvisee advisee = (DistributionAdvisee)bslist.get(i);
-            BridgeServerAdvisor bsa = (BridgeServerAdvisor)advisee.getDistributionAdvisor();
+            CacheServerAdvisor bsa = (CacheServerAdvisor)advisee.getDistributionAdvisor();
             List others = bsa.fetchBridgeServers();
             getLogWriter().info("found these bridgeservers in " + advisee + ": " + others);
             assertEquals(3, others.size());
@@ -799,11 +799,11 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
     vm2.invoke(new SerializableRunnable("Verify bridge server view on " + bsPort2 + " and on " + bsPort4) {
         public void run() {
           Cache c = CacheFactory.getAnyInstance();
-          List bslist = c.getBridgeServers();
+          List bslist = c.getCacheServers();
           assertEquals(2, bslist.size());
           for (int i=0; i < bslist.size(); i++) {
             DistributionAdvisee advisee = (DistributionAdvisee)bslist.get(i);
-            BridgeServerAdvisor bsa = (BridgeServerAdvisor)advisee.getDistributionAdvisor();
+            CacheServerAdvisor bsa = (CacheServerAdvisor)advisee.getDistributionAdvisor();
             List others = bsa.fetchBridgeServers();
             getLogWriter().info("found these bridgeservers in " + advisee + ": " + others);
             assertEquals(3, others.size());
@@ -829,9 +829,9 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
       new SerializableRunnable("stop bridge server") {
           public void run() {
             Cache c = CacheFactory.getAnyInstance();
-            List bslist = c.getBridgeServers();
+            List bslist = c.getCacheServers();
             assertEquals(2, bslist.size());
-            BridgeServer bs = (BridgeServer)bslist.get(0);
+            CacheServer bs = (CacheServer)bslist.get(0);
             bs.stop();
           }
         };
@@ -856,8 +856,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             others = ca.fetchBridgeServers();
             assertEquals(3, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort2) {
                 assertEquals(Arrays.asList(new String[] {"bs2Group1", "bs2Group2"}),
                              Arrays.asList(bsp.getGroups()));
@@ -889,8 +889,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             others = ca.fetchBridgeServers();
             assertEquals(3, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort2) {
                 assertEquals(Arrays.asList(new String[] {"bs2Group1", "bs2Group2"}),
                              Arrays.asList(bsp.getGroups()));
@@ -938,11 +938,11 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
     vm2.invoke(new SerializableRunnable("Verify bridge server saw locator stop") {
         public void run() {
           Cache c = CacheFactory.getAnyInstance();
-          List bslist = c.getBridgeServers();
+          List bslist = c.getCacheServers();
           assertEquals(2, bslist.size());
           for (int i=0; i < bslist.size(); i++) {
             DistributionAdvisee advisee = (DistributionAdvisee)bslist.get(i);
-            BridgeServerAdvisor bsa = (BridgeServerAdvisor)advisee.getDistributionAdvisor();
+            CacheServerAdvisor bsa = (CacheServerAdvisor)advisee.getDistributionAdvisor();
             List others = bsa.fetchControllers();
             assertEquals(1, others.size());
             for (int j=0; j < others.size(); j++) {
@@ -961,7 +961,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
     vm1.invoke(new SerializableRunnable("Verify bridge server saw locator stop") {
         public void run() {
           Cache c = CacheFactory.getAnyInstance();
-          List bslist = c.getBridgeServers();
+          List bslist = c.getCacheServers();
           assertEquals(2, bslist.size());
           for (int i=0; i < bslist.size(); i++) {
             DistributionAdvisee advisee = (DistributionAdvisee)bslist.get(i);
@@ -969,7 +969,7 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
               // skip this one since it is stopped
               continue;
             }
-            BridgeServerAdvisor bsa = (BridgeServerAdvisor)advisee.getDistributionAdvisor();
+            CacheServerAdvisor bsa = (CacheServerAdvisor)advisee.getDistributionAdvisor();
             List others = bsa.fetchControllers();
             assertEquals(1, others.size());
             for (int j=0; j < others.size(); j++) {
@@ -991,9 +991,9 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
           public void run() {
             try {
               Cache c = CacheFactory.getAnyInstance();
-              List bslist = c.getBridgeServers();
+              List bslist = c.getCacheServers();
               assertEquals(2, bslist.size());
-              BridgeServer bs = (BridgeServer)bslist.get(0);
+              CacheServer bs = (CacheServer)bslist.get(0);
               bs.setHostnameForClients("nameForClients");
               bs.start();
             } catch (IOException ex) {
@@ -1016,8 +1016,8 @@ public class GridAdvisorDUnitTest extends DistributedTestCase {
             List others = ca.fetchBridgeServers();
             assertEquals(4, others.size());
             for (int j=0; j < others.size(); j++) {
-              BridgeServerAdvisor.BridgeServerProfile bsp =
-                (BridgeServerAdvisor.BridgeServerProfile)others.get(j);
+              CacheServerAdvisor.CacheServerProfile bsp =
+                (CacheServerAdvisor.CacheServerProfile)others.get(j);
               if (bsp.getPort() == bsPort1) {
                 assertEquals(Arrays.asList(new String[] {"bs1Group1", "bs1Group2"}),
                              Arrays.asList(bsp.getGroups()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HABug36773DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HABug36773DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HABug36773DUnitTest.java
index 5671ac0..e17c77d 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HABug36773DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HABug36773DUnitTest.java
@@ -24,8 +24,8 @@ import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.Connection;
 import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
-import com.gemstone.gemfire.cache.util.BridgeServer;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -272,7 +272,7 @@ public class HABug36773DUnitTest extends DistributedTestCase
     new HABug36773DUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
-    BridgeTestCase.configureConnectionPool(factory, DistributedTestCase.getIPLiteral(), new int[] {PORT1,PORT2}, true, -1, 2, null);
+    ClientServerTestCase.configureConnectionPool(factory, DistributedTestCase.getIPLiteral(), new int[] {PORT1,PORT2}, true, -1, 2, null);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
   }
@@ -287,7 +287,7 @@ public class HABug36773DUnitTest extends DistributedTestCase
     factory.setEarlyAck(true);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HAOverflowMemObjectSizerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
index ea368e0..158848c 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/HAOverflowMemObjectSizerDUnitTest.java
@@ -21,8 +21,8 @@ import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.util.BridgeServer;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -127,7 +127,7 @@ public class HAOverflowMemObjectSizerDUnitTest extends DistributedTestCase {
     RegionAttributes attrs = factory.create();
     Region region = cache.createRegion(regionName, attrs);
     assertNotNull(region);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     assertNotNull(server1);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server1.setPort(port);
@@ -140,7 +140,7 @@ public class HAOverflowMemObjectSizerDUnitTest extends DistributedTestCase {
      * storing capacity controller reference
      */
     cc = ((VMLRURegionMap)((LocalRegion)cache.getRegion(Region.SEPARATOR
-        + BridgeServerImpl.generateNameForClientMsgsRegion(port))).entries)
+        + CacheServerImpl.generateNameForClientMsgsRegion(port))).entries)
         ._getCCHelper();
     return new Integer(server1.getPort());
   }
@@ -160,7 +160,7 @@ public class HAOverflowMemObjectSizerDUnitTest extends DistributedTestCase {
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
     factory.setDataPolicy(DataPolicy.NORMAL);
-    BridgeTestCase.configureConnectionPool(factory, host, port1.intValue(), -1, true, -1, 2, null, -1, -1, false);
+    ClientServerTestCase.configureConnectionPool(factory, host, port1.intValue(), -1, true, -1, 2, null, -1, -1, false);
     RegionAttributes attrs = factory.create();
     Region region = cache.createRegion(regionName, attrs);
     assertNotNull(region);
@@ -227,7 +227,7 @@ public class HAOverflowMemObjectSizerDUnitTest extends DistributedTestCase {
    */
   public static void sizerTestForMemCapacityController(Integer port) {
     region = cache.getRegion(Region.SEPARATOR
-        + BridgeServerImpl.generateNameForClientMsgsRegion(port.intValue()));
+        + CacheServerImpl.generateNameForClientMsgsRegion(port.intValue()));
     assertNotNull(region);
     Set entries = region.entrySet();
     assertTrue(entries.size() > 0);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/Bug51193DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/Bug51193DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/Bug51193DUnitTest.java
index 12f817f..6fab13c 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/Bug51193DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/Bug51193DUnitTest.java
@@ -20,7 +20,7 @@ import com.gemstone.gemfire.cache.execute.ResultCollector;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.tier.ClientHandShake;
 import com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl;
@@ -206,7 +206,7 @@ public class Bug51193DUnitTest extends DistributedTestCase {
     public void execute(FunctionContext context) {
       boolean timeoutMatches = false;
       int expected = (Integer)context.getArguments();
-      AcceptorImpl acceptor = ((BridgeServerImpl) cache.getCacheServers()
+      AcceptorImpl acceptor = ((CacheServerImpl) cache.getCacheServers()
           .get(0)).getAcceptor();
       ServerConnection[] scs = acceptor.getAllServerConnectionList();
       for (int i = 0; i < scs.length; ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/functions/TestFunction.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/functions/TestFunction.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/functions/TestFunction.java
index 445ccd0..3ce15b2 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/functions/TestFunction.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/functions/TestFunction.java
@@ -24,7 +24,7 @@ import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.Assert;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
@@ -927,7 +927,7 @@ public class TestFunction extends FunctionAdapter implements Declarable2 {
         } else {
           r.put("stopped", ++numTimesStopped);
           for (CacheServer s : servers) {
-            if (((BridgeServerImpl)s).getSystem().getDistributedMember().equals(
+            if (((CacheServerImpl)s).getSystem().getDistributedMember().equals(
                 ((GemFireCacheImpl)CacheFactory.getAnyInstance()).getMyId())) {
               s.stop();
               DistributedSystem ds = InternalDistributedSystem.getAnyInstance();
@@ -954,7 +954,7 @@ public class TestFunction extends FunctionAdapter implements Declarable2 {
       }
       
       for (CacheServer s : servers) {
-        if (((BridgeServerImpl)s).getSystem().getDistributedMember().equals(
+        if (((CacheServerImpl)s).getSystem().getDistributedMember().equals(
             ((GemFireCacheImpl)CacheFactory.getAnyInstance()).getMyId())) {
           s.stop();
           DistributedSystem ds = InternalDistributedSystem.getAnyInstance();
@@ -990,7 +990,7 @@ public class TestFunction extends FunctionAdapter implements Declarable2 {
         } else {
           r.put("stopped", ++numTimesStopped);
           for (CacheServer s : servers) {
-            if (((BridgeServerImpl)s).getSystem().getDistributedMember().equals(
+            if (((CacheServerImpl)s).getSystem().getDistributedMember().equals(
                 ((GemFireCacheImpl)CacheFactory.getAnyInstance()).getMyId())) {
               s.stop();
               DistributedSystem ds = InternalDistributedSystem.getAnyInstance();
@@ -1016,7 +1016,7 @@ public class TestFunction extends FunctionAdapter implements Declarable2 {
       }
       
       for (CacheServer s : servers) {
-        if (((BridgeServerImpl)s).getSystem().getDistributedMember().equals(
+        if (((CacheServerImpl)s).getSystem().getDistributedMember().equals(
             ((GemFireCacheImpl)CacheFactory.getAnyInstance()).getMyId())) {
           s.stop();
           DistributedSystem ds = InternalDistributedSystem.getAnyInstance();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java
index b22ac27..b4932a3 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java
@@ -19,9 +19,9 @@ import com.gemstone.gemfire.cache.EntryEvent;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -140,7 +140,7 @@ public class Bug36853EventsExpiryDUnitTest extends CacheTestCase
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
 
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -166,7 +166,7 @@ public class Bug36853EventsExpiryDUnitTest extends CacheTestCase
     new Bug36853EventsExpiryDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
-    BridgeTestCase.configureConnectionPool(factory, hostName, port.intValue(),-1, true, -1, 2, null);
+    ClientServerTestCase.configureConnectionPool(factory, hostName, port.intValue(),-1, true, -1, 2, null);
 
     factory.addCacheListener(new CacheListenerAdapter() {
       public void afterCreate(EntryEvent event)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug48571DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug48571DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug48571DUnitTest.java
index edba084..2113472 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug48571DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug48571DUnitTest.java
@@ -19,7 +19,7 @@ import com.gemstone.gemfire.cache.client.ClientCacheFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
 import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationDUnitTest.java
index 522f8bc..c2e7d85 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/EventIdOptimizationDUnitTest.java
@@ -25,9 +25,9 @@ import com.gemstone.gemfire.cache.RegionEvent;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
@@ -201,7 +201,7 @@ public class EventIdOptimizationDUnitTest extends DistributedTestCase
     for (int i = 0; i < eventIds.length; i++) {
       cache.createRegion(REGION_NAME + i, attrs);
     }
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -226,8 +226,8 @@ public class EventIdOptimizationDUnitTest extends DistributedTestCase
     new EventIdOptimizationDUnitTest("temp").createCache(props);
 
     AttributesFactory factory = new AttributesFactory();
-    BridgeTestCase.configureConnectionPool(factory, hostName, port.intValue(),-1, true, -1, 2, null);
-    final BridgeServer bs1 = cache.addBridgeServer();
+    ClientServerTestCase.configureConnectionPool(factory, hostName, port.intValue(),-1, true, -1, 2, null);
+    final CacheServer bs1 = cache.addCacheServer();
     bs1.setPort(port.intValue());
 
     pool = (PoolImpl)PoolManager.find("testPool");
@@ -248,7 +248,7 @@ public class EventIdOptimizationDUnitTest extends DistributedTestCase
     props.setProperty("locators", "");
     new EventIdOptimizationDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
-    BridgeTestCase.configureConnectionPool(factory, hostName, port.intValue(),-1, true, -1, 2, null);
+    ClientServerTestCase.configureConnectionPool(factory, hostName, port.intValue(),-1, true, -1, 2, null);
     
     factory.setScope(Scope.DISTRIBUTED_ACK);