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:44 UTC

[21/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/main/java/com/gemstone/gemfire/internal/cache/xmlcache/BridgeServerCreation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/BridgeServerCreation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/BridgeServerCreation.java
deleted file mode 100644
index 3a40574..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/BridgeServerCreation.java
+++ /dev/null
@@ -1,249 +0,0 @@
-/*=========================================================================
- * 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.internal.cache.xmlcache;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Set;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.ClientSession;
-import com.gemstone.gemfire.cache.InterestRegistrationListener;
-import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.cache.server.ClientSubscriptionConfig;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.internal.cache.AbstractBridgeServer;
-import com.gemstone.gemfire.internal.cache.InternalCache;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-
-/**
- * Represents a {@link CacheServer} that is created declaratively.
- *
- * @author David Whitlock
- * @since 4.0
- */
-public class BridgeServerCreation extends AbstractBridgeServer {
-
-  // moved to AbstractBridgeServer
-  
-  //////////////////////  Constructors  //////////////////////
-
-  /**
-   * Creates a new <code>BridgeServerCreation</code> with the default
-   * configuration.
-   *
-   * @param cache
-   *        The cache being served
-   */
-  BridgeServerCreation(InternalCache cache) {
-    super(cache);
-  }
-
-  BridgeServerCreation(InternalCache cache, boolean attachListener) {
-    super(cache, attachListener);
-  }
-  
-  /**
-   * Constructor for retaining bridge server information during auto-reconnect
-   * @param cache
-   * @param other
-   */
-  public BridgeServerCreation(InternalCache cache, CacheServer other) {
-    super(cache);
-    setPort(other.getPort());
-    setBindAddress(other.getBindAddress());
-    setHostnameForClients(other.getHostnameForClients());
-    setMaxConnections(other.getMaxConnections());
-    setMaxThreads(other.getMaxThreads());
-    setNotifyBySubscription(other.getNotifyBySubscription());
-    setSocketBufferSize(other.getSocketBufferSize());
-    setTcpNoDelay(other.getTcpNoDelay());
-    setMaximumTimeBetweenPings(other.getMaximumTimeBetweenPings());
-    setMaximumMessageCount(other.getMaximumMessageCount());
-    setMessageTimeToLive(other.getMessageTimeToLive());
-    //      setTransactionTimeToLive(other.getTransactionTimeToLive());  not implemented in CacheServer for v6.6
-    setGroups(other.getGroups());
-    setLoadProbe(other.getLoadProbe());
-    setLoadPollInterval(other.getLoadPollInterval());
-    ClientSubscriptionConfig cscOther = other.getClientSubscriptionConfig();
-    ClientSubscriptionConfig cscThis = this.getClientSubscriptionConfig();
-    // added for configuration of ha overflow
-    cscThis.setEvictionPolicy(cscOther.getEvictionPolicy());
-    cscThis.setCapacity(cscOther.getCapacity());
-    String diskStoreName = cscOther.getDiskStoreName();
-    if (diskStoreName != null) {
-      cscThis.setDiskStoreName(diskStoreName);
-    } else {
-      cscThis.setOverflowDirectory(cscOther.getOverflowDirectory());
-    }
-    // this.cache = null; we should null out the cache since we no longer need it
-  }
-
-  /////////////////////  Instance Methods  /////////////////////
-
-  @Override
-  public void start() throws IOException {
-    // This method is invoked during testing, but it is not necessary
-    // to do anything.
-  }
-
-  @Override
-  public void setNotifyBySubscription(boolean b) {
-    this.notifyBySubscription = b;
-  }
-
-  @Override
-  public boolean getNotifyBySubscription() {
-    return this.notifyBySubscription;
-  }
-
-  @Override
-  public void setSocketBufferSize(int socketBufferSize) {
-    this.socketBufferSize = socketBufferSize;
-  }
-  
-  @Override
-  public int getSocketBufferSize() {
-    return this.socketBufferSize;
-  }
-  
-  @Override
-  public void setTcpNoDelay(boolean setting) {
-    this.tcpNoDelay = setting;
-  }
-  
-  @Override
-  public boolean getTcpNoDelay() {
-    return this.tcpNoDelay;
-  }
-
-  @Override
-  public void setMaximumTimeBetweenPings(int maximumTimeBetweenPings) {
-    this.maximumTimeBetweenPings = maximumTimeBetweenPings;
-  }
-  
-  @Override
-  public int getMaximumTimeBetweenPings() {
-    return this.maximumTimeBetweenPings;
-  }
-  
-  @Override
-  public int getMaximumMessageCount() {
-    return this.maximumMessageCount;
-  }
-
-  @Override
-  public void setMaximumMessageCount(int maximumMessageCount) {
-    this.maximumMessageCount = maximumMessageCount;
-  }
-  
-  @Override
-  public int getMessageTimeToLive() {
-    return this.messageTimeToLive;
-  }
-
-  @Override
-  public void setMessageTimeToLive(int messageTimeToLive) {
-    this.messageTimeToLive = messageTimeToLive;
-  }
-  
-  public boolean isRunning() {
-    throw new UnsupportedOperationException(LocalizedStrings.SHOULDNT_INVOKE.toLocalizedString());
-  }
-
-  public void stop() {
-    throw new UnsupportedOperationException(LocalizedStrings.SHOULDNT_INVOKE.toLocalizedString());
-  }
-
-  /**
-   * Returns whether or not this bridge server has the same
-   * configuration as another bridge server.
-   */
-  @Override
-  public boolean sameAs(CacheServer other) {
-    ClientSubscriptionConfig cscThis = this.getClientSubscriptionConfig();
-    ClientSubscriptionConfig cscOther = other.getClientSubscriptionConfig();
-    boolean result = 
-        this.getPort() == other.getPort() &&
-        this.getSocketBufferSize() == other.getSocketBufferSize() &&
-        this.getMaximumTimeBetweenPings() == other.getMaximumTimeBetweenPings() &&
-        this.getNotifyBySubscription() == other.getNotifyBySubscription() &&
-        this.getMaxConnections() == other.getMaxConnections() &&
-        this.getMaxThreads() == other.getMaxThreads() &&
-        this.getMaximumMessageCount() == other.getMaximumMessageCount() &&
-        this.getMessageTimeToLive() == other.getMessageTimeToLive() &&
-        this.getTcpNoDelay() == other.getTcpNoDelay() &&
-        cscThis.getCapacity() == cscOther.getCapacity() &&
-        cscThis.getEvictionPolicy().equals(cscOther.getEvictionPolicy());
-    String diskStoreName = cscThis.getDiskStoreName();
-    if (diskStoreName != null) {
-      result = result && diskStoreName.equals(cscOther.getDiskStoreName());
-    } else {
-      result = result && cscThis.getOverflowDirectory().equals(cscOther.getOverflowDirectory());
-    }
-    return result;
-  }
-
-  @Override
-  public String toString()
-  {
-    return "BridgeServerCreation on port " + this.getPort() +
-    " notify by subscription " + this.getNotifyBySubscription() +
-    " maximum time between pings " + this.getMaximumTimeBetweenPings() + 
-    " socket buffer size " + this.getSocketBufferSize() + 
-    " maximum connections " + this.getMaxConnections() +
-    " maximum threads " + this.getMaxThreads() +
-    " maximum message count " + this.getMaximumMessageCount() +
-    " message time to live " + this.getMessageTimeToLive() +
-    " groups " + Arrays.asList(getGroups()) +
-    " loadProbe " + loadProbe +
-    " loadPollInterval " + loadPollInterval +
-    this.getClientSubscriptionConfig().toString();
-  }
-  
-  public ClientSubscriptionConfig getClientSubscriptionConfig(){
-    return this.clientSubscriptionConfig;
-  }
-
-  public Set getInterestRegistrationListeners() {
-    //TODO Yogesh : implement me 
-    return null;
-  }
-
-  public void registerInterestRegistrationListener(
-      InterestRegistrationListener listener) {
-    //TODO Yogesh : implement me
-  }
-
-  public void unregisterInterestRegistrationListener(
-      InterestRegistrationListener listener) {
-    //TODO Yogesh : implement me
-  }
-
-  /* (non-Javadoc)
-   * @see com.gemstone.gemfire.cache.util.BridgeServer#getAllClientSessions()
-   */
-  public Set getAllClientSessions() {
-    throw new UnsupportedOperationException("Shouldn't be invoked");
-  }
-
-  /* (non-Javadoc)
-   * @see com.gemstone.gemfire.cache.util.BridgeServer#getClientSession(com.gemstone.gemfire.distributed.DistributedMember)
-   */
-  public ClientSession getClientSession(DistributedMember member) {
-    throw new UnsupportedOperationException("Shouldn't be invoked");
-  }
-
-  /* (non-Javadoc)
-   * @see com.gemstone.gemfire.cache.util.BridgeServer#getClientSession(java.lang.String)
-   */
-  public ClientSession getClientSession(String durableClientId) {
-    throw new UnsupportedOperationException("Shouldn't be invoked");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
index b9fcfe7..0347d67 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheCreation.java
@@ -52,7 +52,6 @@ import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueFactoryImpl
 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.client.internal.BridgePoolImpl;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
@@ -74,7 +73,6 @@ import com.gemstone.gemfire.cache.query.RegionNotFoundException;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.snapshot.CacheSnapshotService;
-import com.gemstone.gemfire.cache.util.BridgeServer;
 import com.gemstone.gemfire.cache.util.GatewayConflictResolver;
 import com.gemstone.gemfire.cache.wan.GatewayReceiver;
 import com.gemstone.gemfire.cache.wan.GatewayReceiverFactory;
@@ -91,7 +89,7 @@ import com.gemstone.gemfire.cache.hdfs.internal.HDFSIntegrationUtil;
 import com.gemstone.gemfire.cache.hdfs.internal.HDFSStoreCreation;
 import com.gemstone.gemfire.cache.hdfs.internal.HDFSStoreFactoryImpl;
 import com.gemstone.gemfire.cache.hdfs.internal.HDFSStoreImpl;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.CacheConfig;
 import com.gemstone.gemfire.internal.cache.CacheServerLauncher;
 import com.gemstone.gemfire.internal.cache.DiskStoreFactoryImpl;
@@ -580,14 +578,14 @@ public class CacheCreation implements InternalCache, Extensible<Cache> {
       }
       
       if (!existingCacheServer) {
-        this.getCacheServers().add(new BridgeServerCreation(cache, false));
+        this.getCacheServers().add(new CacheServerCreation(cache, false));
       }
     }
     
     for (Iterator iter = this.getCacheServers().iterator(); iter.hasNext();) {
-      BridgeServerCreation bridge = (BridgeServerCreation)iter.next();
+      CacheServerCreation bridge = (CacheServerCreation)iter.next();
       
-      BridgeServerImpl impl = (BridgeServerImpl)cache.addCacheServer();
+      CacheServerImpl impl = (CacheServerImpl)cache.addCacheServer();
       impl.configureFrom(bridge);
 
       if (serverPort != null && serverPort != CacheServer.DEFAULT_PORT) {
@@ -604,7 +602,7 @@ public class CacheCreation implements InternalCache, Extensible<Cache> {
       }
       catch (IOException ex) {
         throw new GemFireIOException(
-            LocalizedStrings.CacheCreation_WHILE_STARTING_BRIDGE_SERVER_0
+            LocalizedStrings.CacheCreation_WHILE_STARTING_CACHE_SERVER_0
                 .toLocalizedString(impl), ex);
       }
     }
@@ -680,10 +678,7 @@ public class CacheCreation implements InternalCache, Extensible<Cache> {
         if (drc2 == null) {
           return false;
         }
-        if (!RegionAttributesCreation.equal(drc1.getDiskDir(), drc2.getDiskDir())) {
-          return false;
-        }
-        if (!RegionAttributesCreation.equal(drc1.getBridgeWriter(), drc2.getBridgeWriter())) {
+        if (!drc1.equals(drc2)) {
           return false;
         }
       } else {
@@ -703,12 +698,12 @@ public class CacheCreation implements InternalCache, Extensible<Cache> {
       Collection myBridges = this.getCacheServers();
       Collection otherBridges = other.getCacheServers();
       if (myBridges.size() != otherBridges.size()) {
-        throw new RuntimeException(LocalizedStrings.CacheCreation_BRIDGESERVERS_SIZE.toLocalizedString());
+        throw new RuntimeException(LocalizedStrings.CacheCreation_CACHESERVERS_SIZE.toLocalizedString());
       }
 
       for (Iterator myIter = myBridges.iterator(); myIter.hasNext(); ) {
-        BridgeServerCreation myBridge =
-          (BridgeServerCreation) myIter.next();
+        CacheServerCreation myBridge =
+          (CacheServerCreation) myIter.next();
         boolean found = false;
         for (Iterator otherIter = otherBridges.iterator();
              otherIter.hasNext(); ) {
@@ -720,7 +715,7 @@ public class CacheCreation implements InternalCache, Extensible<Cache> {
         }
 
         if (!found) {
-          throw new RuntimeException(LocalizedStrings.CacheCreation_BRIDGE_0_NOT_FOUND.toLocalizedString(myBridge));
+          throw new RuntimeException(LocalizedStrings.CacheCreation_CACHE_SERVER_0_NOT_FOUND.toLocalizedString(myBridge));
         }
       }
 
@@ -731,22 +726,22 @@ public class CacheCreation implements InternalCache, Extensible<Cache> {
           : PoolManager.getAll();
         int m1Size = m1.size();
         {
-          // ignore any BridgePool instances
+          // ignore any gateway instances
           Iterator it1 = m1.values().iterator();
           while (it1.hasNext()) {
             Pool cp = (Pool)it1.next();
-            if (cp instanceof BridgePoolImpl || ((PoolImpl)cp).isUsedByGateway()) {
+            if (((PoolImpl)cp).isUsedByGateway()) {
               m1Size--;
             }
           }
         }
         int m2Size = m2.size();
         {
-          // ignore any BridgePool instances
+          // ignore any gateway instances
           Iterator it2 = m2.values().iterator();
           while (it2.hasNext()) {
             Pool cp = (Pool)it2.next();
-            if (cp instanceof BridgePoolImpl || ((PoolImpl)cp).isUsedByGateway()) {
+            if (((PoolImpl)cp).isUsedByGateway()) {
               m2Size--;
             }
           }
@@ -770,8 +765,8 @@ public class CacheCreation implements InternalCache, Extensible<Cache> {
           Iterator it1 = m1.values().iterator();
           while (it1.hasNext()) {
             PoolImpl cp = (PoolImpl)it1.next();
-            // ignore any BridgePool instances
-            if (!(cp instanceof BridgePoolImpl) && !(cp).isUsedByGateway()) {
+            // ignore any gateway instances
+            if (!(cp).isUsedByGateway()) {
               cp.sameAs(m2.get(cp.getName()));
             }
           }
@@ -985,17 +980,12 @@ public class CacheCreation implements InternalCache, Extensible<Cache> {
     return (Region)this.roots.get(path);
   }
 
-  @SuppressWarnings("deprecation")
-  public BridgeServer addBridgeServer() {
-    return (BridgeServer)addCacheServer();
-  }
-  
   public CacheServer addCacheServer() {
     return addCacheServer(false);
   }
   
   public CacheServer addCacheServer(boolean isGatewayReceiver) {
-    CacheServer bridge = new BridgeServerCreation(this, false);
+    CacheServer bridge = new CacheServerCreation(this, false);
     this.bridgeServers.add(bridge);
     return bridge;
   }
@@ -1004,9 +994,6 @@ public class CacheCreation implements InternalCache, Extensible<Cache> {
     this.declarablePropertiesMap.put(declarable, properties);
   }
   
-  public List getBridgeServers() {
-    return getCacheServers();
-  }
   public List getCacheServers() {
     return this.bridgeServers;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheServerCreation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheServerCreation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheServerCreation.java
new file mode 100644
index 0000000..d961d14
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheServerCreation.java
@@ -0,0 +1,238 @@
+/*=========================================================================
+ * 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.internal.cache.xmlcache;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.ClientSession;
+import com.gemstone.gemfire.cache.InterestRegistrationListener;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.cache.server.ClientSubscriptionConfig;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.internal.cache.AbstractCacheServer;
+import com.gemstone.gemfire.internal.cache.InternalCache;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+
+/**
+ * Represents a {@link CacheServer} that is created declaratively.
+ *
+ * @author David Whitlock
+ * @since 4.0
+ */
+public class CacheServerCreation extends AbstractCacheServer {
+
+  //////////////////////  Constructors  //////////////////////
+
+  /**
+   * Creates a new <code>BridgeServerCreation</code> with the default
+   * configuration.
+   *
+   * @param cache
+   *        The cache being served
+   */
+  CacheServerCreation(InternalCache cache) {
+    super(cache);
+  }
+
+  CacheServerCreation(InternalCache cache, boolean attachListener) {
+    super(cache, attachListener);
+  }
+  
+  /**
+   * Constructor for retaining bridge server information during auto-reconnect
+   * @param cache
+   * @param other
+   */
+  public CacheServerCreation(InternalCache cache, CacheServer other) {
+    super(cache);
+    setPort(other.getPort());
+    setBindAddress(other.getBindAddress());
+    setHostnameForClients(other.getHostnameForClients());
+    setMaxConnections(other.getMaxConnections());
+    setMaxThreads(other.getMaxThreads());
+    setNotifyBySubscription(other.getNotifyBySubscription());
+    setSocketBufferSize(other.getSocketBufferSize());
+    setTcpNoDelay(other.getTcpNoDelay());
+    setMaximumTimeBetweenPings(other.getMaximumTimeBetweenPings());
+    setMaximumMessageCount(other.getMaximumMessageCount());
+    setMessageTimeToLive(other.getMessageTimeToLive());
+    //      setTransactionTimeToLive(other.getTransactionTimeToLive());  not implemented in CacheServer for v6.6
+    setGroups(other.getGroups());
+    setLoadProbe(other.getLoadProbe());
+    setLoadPollInterval(other.getLoadPollInterval());
+    ClientSubscriptionConfig cscOther = other.getClientSubscriptionConfig();
+    ClientSubscriptionConfig cscThis = this.getClientSubscriptionConfig();
+    // added for configuration of ha overflow
+    cscThis.setEvictionPolicy(cscOther.getEvictionPolicy());
+    cscThis.setCapacity(cscOther.getCapacity());
+    String diskStoreName = cscOther.getDiskStoreName();
+    if (diskStoreName != null) {
+      cscThis.setDiskStoreName(diskStoreName);
+    } else {
+      cscThis.setOverflowDirectory(cscOther.getOverflowDirectory());
+    }
+    // this.cache = null; we should null out the cache since we no longer need it
+  }
+
+  /////////////////////  Instance Methods  /////////////////////
+
+  @Override
+  public void start() throws IOException {
+    // This method is invoked during testing, but it is not necessary
+    // to do anything.
+  }
+
+  @Override
+  public void setNotifyBySubscription(boolean b) {
+    this.notifyBySubscription = b;
+  }
+
+  @Override
+  public boolean getNotifyBySubscription() {
+    return this.notifyBySubscription;
+  }
+
+  @Override
+  public void setSocketBufferSize(int socketBufferSize) {
+    this.socketBufferSize = socketBufferSize;
+  }
+  
+  @Override
+  public int getSocketBufferSize() {
+    return this.socketBufferSize;
+  }
+  
+  @Override
+  public void setTcpNoDelay(boolean setting) {
+    this.tcpNoDelay = setting;
+  }
+  
+  @Override
+  public boolean getTcpNoDelay() {
+    return this.tcpNoDelay;
+  }
+
+  @Override
+  public void setMaximumTimeBetweenPings(int maximumTimeBetweenPings) {
+    this.maximumTimeBetweenPings = maximumTimeBetweenPings;
+  }
+  
+  @Override
+  public int getMaximumTimeBetweenPings() {
+    return this.maximumTimeBetweenPings;
+  }
+  
+  @Override
+  public int getMaximumMessageCount() {
+    return this.maximumMessageCount;
+  }
+
+  @Override
+  public void setMaximumMessageCount(int maximumMessageCount) {
+    this.maximumMessageCount = maximumMessageCount;
+  }
+  
+  @Override
+  public int getMessageTimeToLive() {
+    return this.messageTimeToLive;
+  }
+
+  @Override
+  public void setMessageTimeToLive(int messageTimeToLive) {
+    this.messageTimeToLive = messageTimeToLive;
+  }
+  
+  public boolean isRunning() {
+    throw new UnsupportedOperationException(LocalizedStrings.SHOULDNT_INVOKE.toLocalizedString());
+  }
+
+  public void stop() {
+    throw new UnsupportedOperationException(LocalizedStrings.SHOULDNT_INVOKE.toLocalizedString());
+  }
+
+  /**
+   * Returns whether or not this bridge server has the same
+   * configuration as another bridge server.
+   */
+  @Override
+  public boolean sameAs(CacheServer other) {
+    ClientSubscriptionConfig cscThis = this.getClientSubscriptionConfig();
+    ClientSubscriptionConfig cscOther = other.getClientSubscriptionConfig();
+    boolean result = 
+        this.getPort() == other.getPort() &&
+        this.getSocketBufferSize() == other.getSocketBufferSize() &&
+        this.getMaximumTimeBetweenPings() == other.getMaximumTimeBetweenPings() &&
+        this.getNotifyBySubscription() == other.getNotifyBySubscription() &&
+        this.getMaxConnections() == other.getMaxConnections() &&
+        this.getMaxThreads() == other.getMaxThreads() &&
+        this.getMaximumMessageCount() == other.getMaximumMessageCount() &&
+        this.getMessageTimeToLive() == other.getMessageTimeToLive() &&
+        this.getTcpNoDelay() == other.getTcpNoDelay() &&
+        cscThis.getCapacity() == cscOther.getCapacity() &&
+        cscThis.getEvictionPolicy().equals(cscOther.getEvictionPolicy());
+    String diskStoreName = cscThis.getDiskStoreName();
+    if (diskStoreName != null) {
+      result = result && diskStoreName.equals(cscOther.getDiskStoreName());
+    } else {
+      result = result && cscThis.getOverflowDirectory().equals(cscOther.getOverflowDirectory());
+    }
+    return result;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "BridgeServerCreation on port " + this.getPort() +
+    " notify by subscription " + this.getNotifyBySubscription() +
+    " maximum time between pings " + this.getMaximumTimeBetweenPings() + 
+    " socket buffer size " + this.getSocketBufferSize() + 
+    " maximum connections " + this.getMaxConnections() +
+    " maximum threads " + this.getMaxThreads() +
+    " maximum message count " + this.getMaximumMessageCount() +
+    " message time to live " + this.getMessageTimeToLive() +
+    " groups " + Arrays.asList(getGroups()) +
+    " loadProbe " + loadProbe +
+    " loadPollInterval " + loadPollInterval +
+    this.getClientSubscriptionConfig().toString();
+  }
+  
+  public ClientSubscriptionConfig getClientSubscriptionConfig(){
+    return this.clientSubscriptionConfig;
+  }
+
+  public Set getInterestRegistrationListeners() {
+    //TODO Yogesh : implement me 
+    return null;
+  }
+
+  public void registerInterestRegistrationListener(
+      InterestRegistrationListener listener) {
+    //TODO Yogesh : implement me
+  }
+
+  public void unregisterInterestRegistrationListener(
+      InterestRegistrationListener listener) {
+    //TODO Yogesh : implement me
+  }
+
+  public Set getAllClientSessions() {
+    throw new UnsupportedOperationException("Shouldn't be invoked");
+  }
+
+  public ClientSession getClientSession(DistributedMember member) {
+    throw new UnsupportedOperationException("Shouldn't be invoked");
+  }
+
+  public ClientSession getClientSession(String durableClientId) {
+    throw new UnsupportedOperationException("Shouldn't be invoked");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
index 06eb091..ee4e0ae 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
@@ -85,7 +85,6 @@ import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
 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.client.internal.BridgePoolImpl;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionService;
@@ -95,9 +94,6 @@ import com.gemstone.gemfire.cache.query.internal.index.HashIndex;
 import com.gemstone.gemfire.cache.query.internal.index.PrimaryKeyIndex;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.server.ServerLoadProbe;
-import com.gemstone.gemfire.cache.util.BridgeClient;
-import com.gemstone.gemfire.cache.util.BridgeLoader;
-import com.gemstone.gemfire.cache.util.BridgeWriter;
 import com.gemstone.gemfire.cache.util.ObjectSizer;
 import com.gemstone.gemfire.cache.wan.GatewayEventFilter;
 import com.gemstone.gemfire.cache.wan.GatewayReceiver;
@@ -1223,8 +1219,8 @@ public class CacheXmlGenerator extends CacheXml implements XMLReader {
     if (this.version.compareTo(CacheXmlVersion.VERSION_5_7) < 0) {
       return;
     }
-    if (cp instanceof BridgePoolImpl || ((PoolImpl)cp).isUsedByGateway()) {
-      // no need to generate xml for bridge pools
+    if (((PoolImpl)cp).isUsedByGateway()) {
+      // no need to generate xml for gateway pools
       return;
     }
     AttributesImpl atts = new AttributesImpl();
@@ -1389,12 +1385,6 @@ public class CacheXmlGenerator extends CacheXml implements XMLReader {
         handler.endElement("", DISK_DIR, DISK_DIR);
       }
     }
-    {
-      BridgeWriter bw = cfg.getBridgeWriter();
-      if (bw != null) {
-        generate(CACHE_WRITER, bw);
-      }
-    }
     handler.endElement("", DYNAMIC_REGION_FACTORY, DYNAMIC_REGION_FACTORY);
   }
 
@@ -2194,22 +2184,13 @@ public class CacheXmlGenerator extends CacheXml implements XMLReader {
       }
     }
 
-    if (attrs.getCacheWriter() == attrs.getCacheLoader()
-        && attrs.getCacheWriter() instanceof BridgeClient) {
-      // just do the writer; the single instance will be made both loader and writer
-      if ((!(attrs instanceof RegionAttributesCreation) ||
-           ((RegionAttributesCreation) attrs).hasCacheWriter())) {
-        generate(CACHE_WRITER, attrs.getCacheWriter());
-      }
-    } else {
-      if ((!(attrs instanceof RegionAttributesCreation)
-           || ((RegionAttributesCreation) attrs).hasCacheLoader())) {
-        generate(CACHE_LOADER, attrs.getCacheLoader());
-      }
-      if ((!(attrs instanceof RegionAttributesCreation) ||
-           ((RegionAttributesCreation) attrs).hasCacheWriter())) {
-        generate(CACHE_WRITER, attrs.getCacheWriter());
-      }
+    if ((!(attrs instanceof RegionAttributesCreation)
+        || ((RegionAttributesCreation) attrs).hasCacheLoader())) {
+      generate(CACHE_LOADER, attrs.getCacheLoader());
+    }
+    if ((!(attrs instanceof RegionAttributesCreation) ||
+        ((RegionAttributesCreation) attrs).hasCacheWriter())) {
+      generate(CACHE_WRITER, attrs.getCacheWriter());
     }
     if ((!(attrs instanceof RegionAttributesCreation) ||
          ((RegionAttributesCreation) attrs).hasCacheListeners())) {
@@ -2254,10 +2235,6 @@ public class CacheXmlGenerator extends CacheXml implements XMLReader {
     Properties props = null;
     if (callback instanceof Declarable2) {
       props = ((Declarable2) callback).getConfig();
-    } else if (callback instanceof BridgeWriter) {
-      props = ((BridgeWriter) callback).getProperties();
-    } else if (callback instanceof BridgeLoader) {
-      props = ((BridgeLoader) callback).getProperties();
     } else if (callback instanceof ReflectionBasedAutoSerializer) {
       props = ((ReflectionBasedAutoSerializer) callback).getConfig();
     } else if (callback instanceof Declarable  && cache instanceof GemFireCacheImpl) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
index 9b2f5ed..f0b3612 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
@@ -85,7 +85,6 @@ import com.gemstone.gemfire.cache.query.internal.index.IndexCreationData;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.server.ClientSubscriptionConfig;
 import com.gemstone.gemfire.cache.server.ServerLoadProbe;
-import com.gemstone.gemfire.cache.util.BridgeWriter;
 import com.gemstone.gemfire.cache.util.ObjectSizer;
 import com.gemstone.gemfire.cache.wan.GatewayEventFilter;
 import com.gemstone.gemfire.cache.wan.GatewayEventSubstitutionFilter;
@@ -874,25 +873,10 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
     String poolName = (String)stack.pop();
     String disableRegisterInterest = (String)stack.pop();
     String disablePersistBackup = (String)stack.pop();
-    CacheWriter cw = attrs.getCacheWriter();
-    if(poolName !=null && cw != null) {
-      throw new CacheXmlException("You cannot specify both a poolName and a cacheWriter for a dynamic-region-factory.");
-    }
-    if (cw != null && !(cw instanceof BridgeWriter)) {
-      throw new CacheXmlException(LocalizedStrings.CacheXmlParser_THE_DYNAMICREGIONFACTORY_CACHEWRITER_MUST_BE_AN_INSTANCE_OF_BRIDGEWRITER.toLocalizedString());
-    }
     DynamicRegionFactory.Config cfg;
-    if(poolName != null) {
-      cfg =
-        new DynamicRegionFactory.Config(dir, poolName,
+    cfg = new DynamicRegionFactory.Config(dir, poolName,
             !Boolean.valueOf(disablePersistBackup).booleanValue(),
             !Boolean.valueOf(disableRegisterInterest).booleanValue());
-    } else {
-      cfg =
-        new DynamicRegionFactory.Config(dir, (BridgeWriter)cw,
-          !Boolean.valueOf(disablePersistBackup).booleanValue(),
-          !Boolean.valueOf(disableRegisterInterest).booleanValue());
-    }
     CacheCreation cache = (CacheCreation)stack.peek();
     cache.setDynamicRegionFactoryConfig(cfg);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/ClientCacheCreation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/ClientCacheCreation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/ClientCacheCreation.java
index d12e8ce..83b3ece 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/ClientCacheCreation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/ClientCacheCreation.java
@@ -28,7 +28,6 @@ import com.gemstone.gemfire.cache.client.Pool;
 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.BridgeServer;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PoolFactoryImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -145,10 +144,6 @@ public class ClientCacheCreation extends CacheCreation implements ClientCache {
   }
 
   @Override
-  public BridgeServer addBridgeServer() {
-    throw new UnsupportedOperationException(LocalizedStrings.SHOULDNT_INVOKE.toLocalizedString());
-  }
-  @Override
   public CacheServer addCacheServer() {
     throw new UnsupportedOperationException(LocalizedStrings.SHOULDNT_INVOKE.toLocalizedString());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java
index ff494dc..e740df4 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java
@@ -35,7 +35,6 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.SubscriptionAttributes;
-import com.gemstone.gemfire.cache.util.BridgeClient;
 import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.internal.cache.EvictionAttributesImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -514,10 +513,6 @@ public class RegionAttributesCreation extends UserSpecifiedRegionAttributes impl
     CacheLoader old = this.cacheLoader;
     this.cacheLoader = cacheLoader;
     setHasCacheLoader(true);
-    if (cacheLoader instanceof BridgeClient && !hasCacheWriter()) {
-      // fix for bug 36247
-      setCacheWriter((BridgeClient)cacheLoader);
-    }
     return old;
   }
 
@@ -529,10 +524,6 @@ public class RegionAttributesCreation extends UserSpecifiedRegionAttributes impl
     CacheWriter old = this.cacheWriter;
     this.cacheWriter = cacheWriter;
     setHasCacheWriter(true);
-    if (cacheWriter instanceof BridgeClient && !hasCacheLoader()) {
-      // fix for bug 36247
-      setCacheLoader((BridgeClient)cacheWriter);
-    }
     return old;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
index f5ae3e5..7bf07c9 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
@@ -131,7 +131,7 @@ public class LocalizedStrings extends ParentLocalizedStrings {
   public static final StringId EndpointImpl_0_CAN_NOT_REGISTER_INSTANTIATORS_1_BECAUSE_THE_SERVER_IS_NOT_AVAILABLE = new StringIdImpl(3505, "{0} : Can not register instantiators   {1}  because the server is not available.");
   public static final StringId EndpointImpl_0_CANNOT_UNREGISTER_INTEREST_IN_REGION_1_KEY_2_BECAUSE_THE_SERVER_IS_NOT_AVAILABLE = new StringIdImpl(3506, "{0} : Cannot unregister interest in region  {1}  key  {2}  because the server is not available.");
   public static final StringId EndpointImpl_0_CAN_NOT_NOTIFY_SERVER_THAT_THIS_CLIENT_IS_READY_BECAUSE_THE_SERVER_IS_NOT_AVAILABLE = new StringIdImpl(3507, "{0} : Can not notify server that this client is ready because the server is not available.");
-  public static final StringId CacheCreation_WHILE_STARTING_BRIDGE_SERVER_0 = new StringIdImpl(3508, "While starting bridge server  {0}");
+  public static final StringId CacheCreation_WHILE_STARTING_CACHE_SERVER_0 = new StringIdImpl(3508, "While starting cache server  {0}");
   public static final StringId CacheCreation_WHILE_STARTING_GATEWAY_HUB_0 = new StringIdImpl(3509, "While starting gateway hub  {0}");
   public static final StringId CacheXml_ERROR_WHILE_PARSING_XML = new StringIdImpl(3510, "Error while parsing XML");
   public static final StringId CacheXml_FATAL_ERROR_WHILE_PARSING_XML = new StringIdImpl(3511, "Fatal error while parsing XML");
@@ -654,9 +654,9 @@ public class LocalizedStrings extends ParentLocalizedStrings {
   public static final StringId AdminWaiters_REQUEST_WAIT_WAS_INTERRUPTED = new StringIdImpl(4029, "Request wait was interrupted.");
   public static final StringId AlertLevelChangeMessage_CHANGING_ALERT_LEVEL_TO_0 = new StringIdImpl(4030, "Changing alert level to {0}");
   public static final StringId BridgeServerRequest_ADD_BRIDGE_SERVER = new StringIdImpl(4031, "Add bridge server");
-  public static final StringId BridgeServerRequest_GET_INFO_ABOUT_BRIDGE_SERVER_0 = new StringIdImpl(4032, "Get info about bridge server {0}");
-  public static final StringId BridgeServerRequest_START_BRIDGE_SERVER_0 = new StringIdImpl(4033, "Start bridge server {0}");
-  public static final StringId BridgeServerRequest_STOP_BRIDGE_SERVER_0 = new StringIdImpl(4034, "Stop bridge server {0}");
+  public static final StringId BridgeServerRequest_GET_INFO_ABOUT_BRIDGE_SERVER_0 = new StringIdImpl(4032, "Get info about cache server {0}");
+  public static final StringId BridgeServerRequest_START_BRIDGE_SERVER_0 = new StringIdImpl(4033, "Start cache server {0}");
+  public static final StringId BridgeServerRequest_STOP_BRIDGE_SERVER_0 = new StringIdImpl(4034, "Stop cache server {0}");
   public static final StringId BridgeServerRequest_UNKNOWN_OPERATION_0 = new StringIdImpl(4035, "Unknown operation {0}");
   public static final StringId CacheConfigRequest_SET_A_SINGLE_CACHE_CONFIGURATION_ATTRIBUTE = new StringIdImpl(4036, "Set a single cache configuration attribute");
   public static final StringId CancellationMessage_CANCELLATIONMESSAGE_FROM_0_FOR_MESSAGE_ID_1 = new StringIdImpl(4037, "CancellationMessage from {0} for message id {1}");
@@ -982,7 +982,7 @@ public class LocalizedStrings extends ParentLocalizedStrings {
   public static final StringId StatAlertsManager_STATALERTMANAGER_EVALUATEALERTDEFNSTASK_GENERAL_EXCEPTION_0 = new StringIdImpl(4357, "StatAlertManager.EvaluateAlertDefnsTask :General Exception: {0}");
   public static final StringId CqQueryImpl_CQ_NOT_REGISTERED_ON_PRIMARY = new StringIdImpl(4358, "Cq not registered on primary");
   public static final StringId HACacheServer_INITIALIZED = new StringIdImpl(4359, "Initialized");
-  public static final StringId BridgeServerImpl_CLIENT_MESSAGES_REGION_IS_GETTING_CREATED = new StringIdImpl(4360, "client messages Region is getting created");
+  // ok to reuse 4360
   public static final StringId HARegionQueue_TASK_TO_DECREMENT_THE_REF_COUNT_MAY_NOT_HAVE_BEEN_STARTED = new StringIdImpl(4361, "Exception in HARegionQueue.updateHAContainer(). The task to decrement the ref count by one for all the HAEventWrapper instances of this queue present in the haContainer may not have been started");
   public static final StringId CacheClientNotifier_HACONTAINER_0_IS_NOW_CLEANED_UP = new StringIdImpl(4362, "haContainer ({0}) is now cleaned up.");
   public static final StringId CacheClientNotifier_HACONTAINER_0_HAS_BEEN_CREATED = new StringIdImpl(4363, "haContainer ({0}) has been created.");
@@ -1006,7 +1006,7 @@ public class LocalizedStrings extends ParentLocalizedStrings {
   public static final StringId RegionAttributesCreation_CUSTOMENTRYTIMETOLIVE_IS_NOT_THE_SAME = new StringIdImpl(4381, "CustomEntryTimeToLive is not the same");
   public static final StringId AbstractRegion_CANNOT_SET_CUSTOM_TIME_TO_LIVE_WHEN_STATISTICS_ARE_DISABLED = new StringIdImpl(4382, "Cannot set custom time to live when statistics are disabled");
   public static final StringId PutMessage_UNKNOWN_DESERIALIZATION_POLICY = new StringIdImpl(4383, "unknown deserialization policy");
-  public static final StringId BridgeServerImpl__0_INVALID_EVICTION_POLICY = new StringIdImpl(4384, "{0} Invalid eviction policy");
+  public static final StringId CacheServerImpl__0_INVALID_EVICTION_POLICY = new StringIdImpl(4384, "{0} Invalid eviction policy");
   public static final StringId MergeLogFiles_IF_A_DIRECTORY_IS_SPECIFIED_ALL_LOG_FILES_IN_THAT_DIRECTORY_ARE_MERGED = new StringIdImpl(4385, "If a directory is specified, all .log files in that directory are merged.");
   public static final StringId MergeLogFiles_FILE_0_IS_NEITHER_A_FILE_NOR_A_DIRECTORY = new StringIdImpl(4386, "File ''{0}'' is neither a file nor a directory.");
   public static final StringId AvailablePort_NETWORK_IS_UNREACHABLE = new StringIdImpl(4387, "Network is unreachable");
@@ -1146,9 +1146,9 @@ public class LocalizedStrings extends ParentLocalizedStrings {
   public static final StringId Oplog_OPLOG_SHUTDOWN_INTERRUPTED_ACQUIRING_READ_LOCK = new StringIdImpl(4524, "Oplog#shutdown: interrupted acquiring read lock.");
   public static final StringId InternalDistributedSystem_UNABLE_TO_START_THE_MONITOR_SERVICE = new StringIdImpl(4525, "Unable to start the monitor service");
   public static final StringId AbstractDistributionConfig_REDIS_BIND_ADDRESS_0_INVALID_MUST_BE_IN_1 = new StringIdImpl(4526, "The redis-bind-address \"{0}\" is not a valid address for this machine.  These are the valid addresses for this machine: {1}");
-  public static final StringId BridgeServerImpl_CACHESERVER_ERROR_CLOSING_LOAD_MONITOR = new StringIdImpl(4527, "CacheServer - Error closing load monitor");
-  public static final StringId BridgeServerImpl_CACHESERVER_ERROR_CLOSING_ADVISOR = new StringIdImpl(4528, "CacheServer - Error closing advisor");
-  public static final StringId BridgeServerImpl_CACHESERVER_ERROR_CLOSING_ACCEPTOR_MONITOR = new StringIdImpl(4529, "CacheServer - Error closing acceptor monitor");
+  public static final StringId CacheServerImpl_CACHESERVER_ERROR_CLOSING_LOAD_MONITOR = new StringIdImpl(4527, "CacheServer - Error closing load monitor");
+  public static final StringId CacheServerImpl_CACHESERVER_ERROR_CLOSING_ADVISOR = new StringIdImpl(4528, "CacheServer - Error closing advisor");
+  public static final StringId CacheServerImpl_CACHESERVER_ERROR_CLOSING_ACCEPTOR_MONITOR = new StringIdImpl(4529, "CacheServer - Error closing acceptor monitor");
   public static final StringId DiskRegion_COMPLEXDISKREGION_CLOSE_EXCEPTION_IN_STOPPING_COMPACTOR = new StringIdImpl(4530, "DiskRegion::close: Exception in stopping compactor");
   public static final StringId CacheClientNotifier_CACHECLIENTNOTIFIER_CAUGHT_EXCEPTION_ATTEMPTING_TO_CLIENT = new StringIdImpl(4531, "CacheClientNotifier: Caught exception attempting to client: ");
   public static final StringId CqQueryImpl_EXCEPTION_WHILE_EXECUTING_CQ_EXCEPTION_0 = new StringIdImpl(4532, "Exception while executing cq Exception: {0}");
@@ -1296,7 +1296,7 @@ public class LocalizedStrings extends ParentLocalizedStrings {
   public static final StringId GatewayImpl_PAUSED__0 = new StringIdImpl(4674, "Paused {0}");
   public static final StringId GatewayImpl_RESUMED__0 = new StringIdImpl(4675, "Resumed {0}");
   public static final StringId CacheClientProxy_NOT_PRIMARY = new StringIdImpl(4676, "This process is not the primary server for the given client");
-  public static final StringId BridgeServerImpl_MUST_BE_RUNNING = new StringIdImpl(4677, "The bridge server must be running to use this operation");
+  public static final StringId CacheServerImpl_MUST_BE_RUNNING = new StringIdImpl(4677, "The cache server must be running to use this operation");
   public static final StringId InitialImageOperation_0_UNABLE_TO_FLUSH_STATE_TO_1 = new StringIdImpl(4678, "{0}: Unable to flush state to {1} for concurrent gii union");
   public static final StringId InstantiatorRecoveryListener_INSTANTIATORRECOVERYTASK_ERROR_CLASSNOTFOUNDEXCEPTION = new StringIdImpl(4679, "InstantiatorRecoveryTask - Error ClassNotFoundException: {0}");
   public static final StringId AbstractRegion_NO_CUSTOM_EVICTION_SET = new StringIdImpl(4680, "Custom eviction not enabled for region {0}");  public static final StringId ResourceAdvisor_MEMBER_CAUGHT_EXCEPTION_PROCESSING_PROFILE = new StringIdImpl(4682, "This member caught exception processing profile {0} {1}");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
index 21fc52e..44e67ca 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
@@ -41,13 +41,13 @@ class ParentLocalizedStrings {
   public static final StringId AbstractHealthEvaluator_POOR_HEALTH__0 = new StringIdImpl(1026, "POOR_HEALTH:  {0}");
   public static final StringId AbstractRegion_CACHECALLBACK_CLOSE_EXCEPTION = new StringIdImpl(1027, "CacheCallback close exception");
   // ok to reuse 1028
-  public static final StringId AcceptorImpl_BRIDGE_SERVER_CONNECTION_LISTENER_BOUND_TO_ADDRESS_0_WITH_BACKLOG_1 = new StringIdImpl(1029, "Bridge server connection listener bound to address {0} with backlog {1}.");
-  public static final StringId AcceptorImpl_BRIDGE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION_DUE_TO_SOCKET_TIMEOUT = new StringIdImpl(1030, "Bridge server: failed accepting client connection due to socket timeout.");
-  public static final StringId AcceptorImpl_BRIDGE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION__0 = new StringIdImpl(1031, "Bridge server: failed accepting client connection  {0}");
-  public static final StringId AcceptorImpl_BRIDGE_SERVER_ON_PORT_0_IS_SHUTTING_DOWN = new StringIdImpl(1032, "Bridge server on port {0} is shutting down.");
-  public static final StringId AcceptorImpl_BRIDGE_SERVER_TIMED_OUT_WAITING_FOR_HANDSHAKE_FROM__0 = new StringIdImpl(1033, "Bridge server: timed out waiting for handshake from  {0}");
-  public static final StringId AcceptorImpl_BRIDGE_SERVER_UNEXPECTED_EXCEPTION = new StringIdImpl(1034, "Bridge server: Unexpected Exception");
-  public static final StringId AcceptorImpl_BRIDGE_SERVER_UNEXPECTED_IOEXCEPTION_FROM_ACCEPT = new StringIdImpl(1035, "Bridge server: Unexpected IOException from accept");
+  public static final StringId AcceptorImpl_CACHE_SERVER_CONNECTION_LISTENER_BOUND_TO_ADDRESS_0_WITH_BACKLOG_1 = new StringIdImpl(1029, "Cache server connection listener bound to address {0} with backlog {1}.");
+  public static final StringId AcceptorImpl_CACHE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION_DUE_TO_SOCKET_TIMEOUT = new StringIdImpl(1030, "Cache server: failed accepting client connection due to socket timeout.");
+  public static final StringId AcceptorImpl_CACHE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION__0 = new StringIdImpl(1031, "Cache server: failed accepting client connection  {0}");
+  public static final StringId AcceptorImpl_CACHE_SERVER_ON_PORT_0_IS_SHUTTING_DOWN = new StringIdImpl(1032, "Cache server on port {0} is shutting down.");
+  public static final StringId AcceptorImpl_CACHE_SERVER_TIMED_OUT_WAITING_FOR_HANDSHAKE_FROM__0 = new StringIdImpl(1033, "Cache server: timed out waiting for handshake from  {0}");
+  public static final StringId AcceptorImpl_CACHE_SERVER_UNEXPECTED_EXCEPTION = new StringIdImpl(1034, "Cache server: Unexpected Exception");
+  public static final StringId AcceptorImpl_CACHE_SERVER_UNEXPECTED_IOEXCEPTION_FROM_ACCEPT = new StringIdImpl(1035, "Cache server: Unexpected IOException from accept");
   public static final StringId AcceptorImpl_EXCEEDED_MAX_CONNECTIONS_0 = new StringIdImpl(1036, "exceeded max-connections {0}");
   public static final StringId AcceptorImpl_IGNORING = new StringIdImpl(1037, "ignoring");
   public static final StringId AcceptorImpl_IGNORING_EVENT_ON_SELECTOR_KEY__0 = new StringIdImpl(1038, "ignoring event on selector key  {0}");
@@ -132,16 +132,10 @@ class ParentLocalizedStrings {
   public static final StringId BaseCommand_UNKNOWN_QUERY_EXCEPTION = new StringIdImpl(1117, "Uknown query Exception.");
   public static final StringId BaseCommand_SEVERE_CACHE_EXCEPTION_0 = new StringIdImpl(1118, "Severe cache exception : {0}");
   public static final StringId BaseCommand_UNEXPECTED_QUERYINVALIDEXCEPTION_WHILE_PROCESSING_QUERY_0 = new StringIdImpl(1119, "Unexpected QueryInvalidException while processing query {0}");
-  public static final StringId BridgeLoader_THE_BRIDGELOADER_COULD_NOT_ACQUIRE_OR_CREATE_A_CONNECTION_BECAUSE__0 = new StringIdImpl(1120, "The BridgeLoader could not acquire or create a Connection because  {0}");
-  public static final StringId BridgeLoader_THE_SERVER_IS_UNREACHABLE_COULD_NOT_CONNECT_AFTER_0_ATTEMPTS = new StringIdImpl(1121, "The Server is unreachable, could not connect after {0} attempts.");
-  public static final StringId BridgeServerImpl_CACHESERVER_CONFIGURATION___0 = new StringIdImpl(1122, "CacheServer Configuration:   {0}");
-  public static final StringId BridgeServerImpl_FORCING_NOTIFYBYSUBSCRIPTION_TO_SUPPORT_DYNAMIC_REGIONS = new StringIdImpl(1123, "Forcing notifyBySubscription to support dynamic regions");
-  public static final StringId BridgeServerImpl_STARTED__0 = new StringIdImpl(1124, "Started  {0}");
-  public static final StringId BridgeWriter_0_NO_ACTIVE_SERVERS_WERE_FOUND = new StringIdImpl(1126, "{0}: No active servers were found.");
-  public static final StringId BridgeWriter_0_NO_AVAILABLE_CONNECTION_WAS_FOUND_BUT_THE_FOLLOWING_ACTIVE_SERVERS_EXIST_1 = new StringIdImpl(1127, "{0}: No available connection was found, but the following active server(s) exist: {1}");
-  public static final StringId BridgeWriter_0_THE_BRIDGEWRITER_HAS_BEEN_CLOSED = new StringIdImpl(1128, "{0}: The BridgeWriter has been closed.");
-  public static final StringId BridgeWriter_THE_BRIDGEWRCLEARITER_COULD_NOT_ACQUIRE_OR_CREATE_A_CONNECTION_BECAUSE_0 = new StringIdImpl(1129, "The BridgeWrcleariter could not acquire or create a Connection because: {0}");
-  public static final StringId BridgeWriter_THE_BRIDGEWRITER_COULD_NOT_ACQUIRE_OR_CREATE_A_CONNECTION_BECAUSE__0 = new StringIdImpl(1130, "The BridgeWriter could not acquire or create a Connection because:  {0}");
+  // ok to reuse 1120..1121
+  public static final StringId CacheServerImpl_CACHESERVER_CONFIGURATION___0 = new StringIdImpl(1122, "CacheServer Configuration:   {0}");
+  public static final StringId CacheServerImpl_FORCING_NOTIFYBYSUBSCRIPTION_TO_SUPPORT_DYNAMIC_REGIONS = new StringIdImpl(1123, "Forcing notifyBySubscription to support dynamic regions");
+  // ok to reuse 1124..1130
   public static final StringId BucketAdvisor_ATTEMPTED_TO_CLOSE_BUCKETADVISOR_THAT_IS_ALREADY_CLOSED = new StringIdImpl(1131, "Attempted to close BucketAdvisor that is already CLOSED");
   public static final StringId AgentImpl_COULD_NOT_TAIL_0_BECAUSE_1 = new StringIdImpl(1132, "Could not tail \"{0}\" because: {1}");
   public static final StringId SystemAdmin_USED_TO_SPECIFY_A_HOST_NAME_OR_IP_ADDRESS_TO_GIVE_TO_CLIENTS_SO_THEY_CAN_CONNECT_TO_A_LOCATOR = new StringIdImpl(1133, "Used to specify a host name or IP address to give to clients so they can connect to a locator.");
@@ -333,10 +327,10 @@ class ParentLocalizedStrings {
   public static final StringId CqQueryImpl_REGION__0_SPECIFIED_WITH_CQ_NOT_FOUND_CQNAME_1 = new StringIdImpl(1319, "Region : {0} specified with cq not found. CqName: {1}");
   public static final StringId CqQueryImpl_RUNTIMEEXCEPTION_OCCOURED_IN_THE_CQLISTENER_OF_THE_CQ_CQNAME_0_ERROR_1 = new StringIdImpl(1320, "RuntimeException occurred in the CqListener of the CQ, CqName : {0} Error : {1}");
   public static final StringId CqQueryImpl_SELECT_DISTINCT_QUERIES_NOT_SUPPORTED_IN_CQ = new StringIdImpl(1321, "select DISTINCT queries not supported in CQ");
-  public static final StringId CqQueryImpl_THE_ESTABLISHCALLBACKCONNECTION_ON_BRIDGEWRITER_CLIENT_INSTALLED_ON_REGION_0_IS_SET_TO_FALSE = new StringIdImpl(1322, "The ''establishCallbackConnection'' on BridgeWriter/Client installed on Region {0} is set to false.");
+  // ok to reuse 1322
   public static final StringId CqQueryImpl_THE_WHERE_CLAUSE_IN_CQ_QUERIES_CANNOT_REFER_TO_A_REGION = new StringIdImpl(1323, "The WHERE clause in CQ queries cannot refer to a region");
   public static final StringId CqQueryImpl_UNABLE_TO_CREATE_CQ_0_ERROR__1 = new StringIdImpl(1324, "Unable to create cq {0} Error : {1}");
-  public static final StringId CqQueryImpl_UNABLE_TO_GET_THE_CONNECTIONPROXY_THE_REGION_MAY_NOT_HAVE_A_BRIDGEWRITER_OR_BRIDGECLIENT_INSTALLED_ON_IT = new StringIdImpl(1325, "Unable to get the connectionProxy. The Region may not have BridgeWriter or BridgeClient installed on it.");
+  // ok to reuse 1325
   public static final StringId CqQueryImpl_UNSUPPORTED_SEND_REQUEST_TO_SERVER = new StringIdImpl(1326, "Unsupported send request to Server.");
   public static final StringId CqQueryImpl_CQ_IS_IN_RUNNING_STATE_CQNAME_0 = new StringIdImpl(1327, "CQ is in running state, CqName : {0}");
   public static final StringId CqService_0_FAILED_TO_GET_THE_SPECIFIED_CQ_1 = new StringIdImpl(1328, "{0}: Failed to get the specified CQ: {1}");
@@ -679,7 +673,7 @@ class ParentLocalizedStrings {
   public static final StringId IndexCreationMsg_REGION_IS_LOCALLY_DESTROYED_THROWING_REGIONDESTROYEDEXCEPTION_FOR__0 = new StringIdImpl(1673, "Region is locally destroyed, throwing RegionDestroyedException for  {0}");
   public static final StringId CqQueryImpl_FAILED_TO_STOP_THE_CQ_CQNAME_0_THE_SERVER_ENDPOINTS_ON_WHICH_THIS_CQ_WAS_REGISTERED_WERE_NOT_FOUND = new StringIdImpl(1676, "Failed to stop the cq. CqName: {0}. The server endpoints on which this cq was registered were not found.");
   public static final StringId InitialImageOperation_IGNORING_EXCEPTION_DURING_GETFROMALL = new StringIdImpl(1677, "Ignoring exception during getFromAll");
-  public static final StringId InternalBridgeMembership_DONOTIFYJOINED_HAS_NO_LISTENERS_TO_NOTIFY_FOR__0 = new StringIdImpl(1678, "doNotifyJoined has no listeners to notify for  {0}");
+  // ok to reuse 1678
   public static final StringId InternalDataSerializer_COULD_NOT_LOAD_DATASERIALIZER_CLASS_0 = new StringIdImpl(1679, "Could not load DataSerializer class: {0}");
   public static final StringId InternalDataSerializer_REGISTER_DATASERIALIZER_0_OF_CLASS_1 = new StringIdImpl(1680, "Register DataSerializer {0} of class {1}");
   public static final StringId InternalDistributedSystem_CONNECTLISTENER_THREW = new StringIdImpl(1681, "ConnectListener threw...");
@@ -919,7 +913,7 @@ class ParentLocalizedStrings {
   public static final StringId PartitionedRegion_WILL_BE_CREATING_INDEX_ON_THIS_VM_BECAUSE_OF_CREATEINDEX_MESSAGE_COMING_FROM_REMOTE_VM_WITH_THESE_SETTINGS_0 = new StringIdImpl(1909, "Will be creating index on this vm because of createIndex message coming from remote vm with these settings {0} ");
   public static final StringId PartitionedRegion_WILL_BE_REMOVING_ALL_THE_BUCKET_INDEXES = new StringIdImpl(1910, "Will be removing all the bucket indexes");
   public static final StringId PartitionedRegion_WILL_BE_REMOVING_INDEXES_ON___0__BUCKETS = new StringIdImpl(1911, "Will be removing indexes on :  {0}  buckets");
-  public static final StringId PartitionedRegion_WILL_SEND_UPDATED_PROFILE_NOW_FOR_REQUIRING_OLD_VALUE_ON_THIS_BRIDGE_SERVER = new StringIdImpl(1912, "Will send updated profile now for requiring old value on this bridge server.");
+  // ok to reuse 1912
   public static final StringId ProcessBatch_0_CAUGHT_EXCEPTION_PROCESSING_BATCH_CREATE_REQUEST_1_FOR_2_EVENTS = new StringIdImpl(1913, "{0}: Caught exception processing batch create request {1} for {2} events");
   public static final StringId ProcessBatch_0_CAUGHT_EXCEPTION_PROCESSING_BATCH_DESTROY_REQUEST_1_CONTAINING_2_EVENTS = new StringIdImpl(1914, "{0}: Caught exception processing batch destroy request {1} containing {2} events");
   public static final StringId ProcessBatch_0_CAUGHT_EXCEPTION_PROCESSING_BATCH_REQUEST_1_CONTAINING_2_EVENTS = new StringIdImpl(1915, "{0}: Caught exception processing batch request {1} containing {2} events");
@@ -994,7 +988,7 @@ class ParentLocalizedStrings {
   public static final StringId Request_THE_INPUT_REGION_NAME_FOR_THE_GET_REQUEST_IS_NULL = new StringIdImpl(1984, "The input region name for the get request is null.");
   public static final StringId RuntimeDistributionConfigImpl_STILL_USING_PREVIOUS_LICENSE_BECAUSE_A_VALID_LICENSE_WAS_NOT_FOUND_AFTER_CHANGING_THE_LICENSEFILE__0 = new StringIdImpl(1985, "Still using previous license because a valid license was not found after changing the \"license-file\".  {0}");
   public static final StringId RuntimeDistributionConfigImpl_STILL_USING_PREVIOUS_LICENSE_BECAUSE_A_VALID_LICENSE_WAS_NOT_FOUND_AFTER_CHANGING_THE_LICENSETYPE__0 = new StringIdImpl(1986, "Still using previous license because a valid license was not found after changing the \"license-type\".  {0}");
-  public static final StringId LocalRegion_UNHANDLED_EXCEPTION_CAUGHT_WHILE_PROCESSING_BRIDGE_FILTERS = new StringIdImpl(1987, "Unhandled exception caught while processing bridge filters");
+  // ok to reuse 1987
   public static final StringId ServerConnection_0_HANDSHAKE_ACCEPT_FAILED_ON_SOCKET_1_2 = new StringIdImpl(1988, "{0}: Handshake accept failed on socket {1}: {2}");
   public static final StringId ServerConnection_0_HANDSHAKE_REPLY_CODE_TIMEOUT_NOT_RECEIVED_WITH_IN_1_MS = new StringIdImpl(1989, "{0}: Handshake reply code timeout, not received with in {1} ms");
   public static final StringId ServerConnection_0_RECEIVED_NO_HANDSHAKE_REPLY_CODE = new StringIdImpl(1990, "{0}: Received no handshake reply code");
@@ -1042,7 +1036,7 @@ class ParentLocalizedStrings {
   public static final StringId RemoteMessage_REGION_0_NOT_COLOCATED_WITH_TRANSACTION = new StringIdImpl(2026, "Region {0} not colocated with other regions in transaction");
   public static final StringId ServerConnection_BATCH_IDS_ARE_OUT_OF_ORDER_SETTING_LATESTBATCHID_TO_0_IT_WAS_1 = new StringIdImpl(2027, "Batch IDs are out of order. Setting latestBatchId to:{0}. It was:{1}");
   public static final StringId DistributionManager_Cache_Time_Offset_Skew_Warning = new StringIdImpl(2028, "New cache time offset calculated is off more than {0} ms from earlier offset.");
-  public static final StringId ServerConnection_LICENSE_DETAILS_CURRENT_BRIDGE_CONNECTIONS_SIZE_0_VALUES_1 = new StringIdImpl(2029, "License details: current bridge connections (size {0}) values = {1}");
+  // ok to reuse 2029
   public static final StringId CacheXmlParser_UNKNOWN_INDEX_TYPE = new StringIdImpl(2030, "Unknown index type defined as {0}, will be set to range index");
   public static final StringId TXStateStub_LOCAL_DESTROY_NOT_ALLOWED_IN_TRANSACTION = new StringIdImpl(2031, "localDestroy() is not allowed in a transaction");
   public static final StringId TXStateStub_LOCAL_INVALIDATE_NOT_ALLOWED_IN_TRANSACTION = new StringIdImpl(2032, "localInvalidate() is not allowed in a transaction");
@@ -1290,19 +1284,9 @@ class ParentLocalizedStrings {
   public static final StringId BaseCommand_UNKNOWN_RESULT_TYPE_0 = new StringIdImpl(2269, "Unknown result type:  {0}");
   public static final StringId BaseRecordManager_NAME_DIRECTORY_MUST_EXIST = new StringIdImpl(2270, "Name directory must exist");
   public static final StringId BaseRecordManager_RECORDMANAGER_HAS_BEEN_CLOSED = new StringIdImpl(2271, "RecordManager has been closed");
-  public static final StringId BridgeLoader_ALREADY_INITIALIZED = new StringIdImpl(2272, "Already initialized");
-  public static final StringId BridgeLoader_INVALID_LOAD_PARAMS_CHECK_HELPER_OBJECT = new StringIdImpl(2273, "Invalid load params, check helper object");
-  public static final StringId BridgeLoader_THE_BRIDGELOADER_HAS_BEEN_CLOSED = new StringIdImpl(2274, "The BridgeLoader has been closed.");
-  public static final StringId BridgeServerImpl_A_BRIDGE_SERVERS_CONFIGURATION_CANNOT_BE_CHANGED_ONCE_IT_IS_RUNNING = new StringIdImpl(2275, "A bridge server''s configuration cannot be changed once it is running.");
-  public static final StringId BridgeWriter_ALREADY_INITIALIZED = new StringIdImpl(2276, "Already initialized");
-  public static final StringId BridgeWriter_INTEREST_REGISTRATION_REQUIRES_ESTABLISHCALLBACKCONNECTION_PARAMETER_TO_BE_SET_TO_TRUE = new StringIdImpl(2277, "Interest registration requires establishCallbackConnection parameter to be set to true.");
-  public static final StringId BridgeWriter_INVALID_CREATE_PARAMETERS_CHECK_ENTRY_EVENT_OBJECT = new StringIdImpl(2278, "Invalid create parameters. Check entry event object");
-  public static final StringId BridgeWriter_INVALID_KEYSET_PARAMETERS_THE_REGION_CANNOT_BE_NULL = new StringIdImpl(2279, "Invalid keySet parameters. The region cannot be null.");
-  public static final StringId BridgeWriter_INVALID_REGION_PARAMS = new StringIdImpl(2280, "Invalid region params");
-  public static final StringId BridgeWriter_THE_BRIDGEWRITER_HAS_BEEN_CLOSED = new StringIdImpl(2281, "The BridgeWriter has been closed.");
-  public static final StringId BridgeWriter_THE_INPUT_KEY_CANNOT_BE_NULL = new StringIdImpl(2282, "The input key cannot be null");
-  public static final StringId BridgeWriter_THE_INPUT_PARENT_REGION_NAME_0_IS_INVALID = new StringIdImpl(2283, "The input parent region name ( {0} ) is invalid");
-  public static final StringId BridgeWriter_THE_INPUT_REGION_NAME_0_IS_INVALID = new StringIdImpl(2284, "The input region name ( {0} ) is invalid");
+  // ok to reuse 2272..2274
+  public static final StringId CacheServerImpl_A_CACHE_SERVERS_CONFIGURATION_CANNOT_BE_CHANGED_ONCE_IT_IS_RUNNING = new StringIdImpl(2275, "A cache server''s configuration cannot be changed once it is running.");
+  // ok to reuse 2276..2284
   public static final StringId BucketAdvisor_CANNOT_CHANGE_FROM_0_TO_1 = new StringIdImpl(2285, "Cannot change from  {0}  to  {1}");
   public static final StringId BucketRegion_THIS_SHOULD_NEVER_BE_CALLED_ON_0 = new StringIdImpl(2286, "This should never be called on  {0}");
   public static final StringId BucketSizeMessage_FAILED_SENDING_0 = new StringIdImpl(2287, "Failed sending < {0} >");
@@ -1321,8 +1305,8 @@ class ParentLocalizedStrings {
   public static final StringId CacheCollector_UNABLE_TO_MIX_REGION_AND_ENTRY_SNAPSHOTS_IN_CACHECOLLECTOR = new StringIdImpl(2300, "Unable to mix region and entry snapshots in CacheCollector.");
   public static final StringId CacheCreation_ATTRIBUTES_FOR_0_DO_NOT_MATCH = new StringIdImpl(2301, "Attributes for  {0}  do not match");
   //ok to reuse 2302,2303
-  public static final StringId CacheCreation_BRIDGESERVERS_SIZE = new StringIdImpl(2304, "bridgeServers size");
-  public static final StringId CacheCreation_BRIDGE_0_NOT_FOUND = new StringIdImpl(2305, "bridge  {0}  not found");
+  public static final StringId CacheCreation_CACHESERVERS_SIZE = new StringIdImpl(2304, "cacheServers size");
+  public static final StringId CacheCreation_CACHE_SERVER_0_NOT_FOUND = new StringIdImpl(2305, "cache server {0} not found");
   public static final StringId CacheCreation_NAMEDATTRIBUTES_SIZE = new StringIdImpl(2306, "namedAttributes size");
   public static final StringId CacheCreation_NO_ATTRIBUTES_FOR_0 = new StringIdImpl(2307, "No attributes for  {0}");
   public static final StringId CacheCreation_NO_ROOT_0 = new StringIdImpl(2308, "no root  {0}");
@@ -1374,7 +1358,7 @@ class ParentLocalizedStrings {
   public static final StringId CacheXmlParser_CLASS_0_IS_NOT_AN_INSTANCE_OF_DECLARABLE = new StringIdImpl(2354, "Class \"{0}\" is not an instance of Declarable.");
   public static final StringId CacheXmlParser_NO_CACHE_ELEMENT_SPECIFIED = new StringIdImpl(2355, "No cache element specified.");
   public static final StringId CacheXmlParser_ONLY_A_PARAMETER_IS_ALLOWED_IN_THE_CONTEXT_OF_0 = new StringIdImpl(2356, "Only a parameter is allowed in the context of  {0}");
-  public static final StringId CacheXmlParser_THE_DYNAMICREGIONFACTORY_CACHEWRITER_MUST_BE_AN_INSTANCE_OF_BRIDGEWRITER = new StringIdImpl(2357, "The dynamic-region-factory cache-writer must be an instance of BridgeWriter.");
+  // ok to reuse 2357
   public static final StringId CacheXmlParser_UNKNOWN_DATA_POLICY_0 = new StringIdImpl(2358, "Unknown data policy:  {0}");
   public static final StringId CacheXmlParser_UNKNOWN_EXPIRATION_ACTION_0 = new StringIdImpl(2359, "Unknown expiration action:  {0}");
   public static final StringId CacheXmlParser_UNKNOWN_GATEWAY_HUB_POLICY_0 = new StringIdImpl(2360, "Unknown gateway hub policy:  {0}");
@@ -1628,7 +1612,7 @@ class ParentLocalizedStrings {
   public static final StringId DummyQRegion_NOT_YET_IMPLEMENTED = new StringIdImpl(2610, "Not yet implemented");
   public static final StringId DumpB2NRegion_WAITED_TOO_LONG_FOR_REGION_TO_INITIALIZE_0 = new StringIdImpl(2611, "Waited too long for region to initialize {0}");
   public static final StringId DynamicRegionFactory_DYNAMIC_REGION_0_HAS_NOT_BEEN_CREATED = new StringIdImpl(2612, "Dynamic region \"{0}\" has not been created.");
-  public static final StringId DynamicRegionFactory_THE_CLIENT_POOL_OF_A_DYNAMICREGIONFACTORY_MUST_BE_CONFIGURED_WITH_ESTABLISHCALLBACKCONNECTION_SET_TO_TRUE = new StringIdImpl(2613, "The client pool of a DynamicRegionFactory must be configured with establishCallbackConnection set to true.");
+  // ok to reuse 2613
   public static final StringId ElderState_CANNOT_FORCE_GRANTOR_RECOVERY_FOR_GRANTOR_THAT_IS_TRANSFERRING = new StringIdImpl(2614, "Cannot force grantor recovery for grantor that is transferring");
   public static final StringId EndpointImpl_0_ALL_EXISTING_CONNECTIONS_TO_THIS_SERVER_ARE_IN_USE = new StringIdImpl(2615, "{0} : all existing connections to this server are in use");
   public static final StringId EndpointImpl_0_THE_CACHESERVER_HAS_DIED_UNEXPECTEDLY = new StringIdImpl(2616, "{0} : The CacheServer has died unexpectedly");
@@ -1686,7 +1670,7 @@ class ParentLocalizedStrings {
   public static final StringId GatewayHubCreation_NOT_SUPPORTED = new StringIdImpl(2669, "Not supported");
   public static final StringId GatewayHubCreation_REMOVEGATEWAY_IS_NOT_SUPPORTED = new StringIdImpl(2670, "removeGateway is not supported");
   public static final StringId GatewayHubImpl_AN_UNKNOWN_GATEWAY_HUB_POLICY_0_WAS_SPECIFIED_IT_MUST_BE_ONE_OF_1_2_3 = new StringIdImpl(2671, "An unknown gateway hub policy ( {0} ) was specified. It must be one of < {1}   {2}   {3} >.");
-  public static final StringId GatewayHubImpl_A_BRIDGE_SERVERS_CONFIGURATION_CANNOT_BE_CHANGED_ONCE_IT_IS_RUNNING = new StringIdImpl(2672, "A bridge server''s configuration cannot be changed once it is running.");
+  // ok to reuse 2672
   public static final StringId GatewayHubImpl_GATEWAYHUB_0_ALREADY_DEFINES_A_GATEWAY_WITH_ID_1 = new StringIdImpl(2673, "GatewayHub  {0}  already defines a Gateway with id= {1}");
   public static final StringId GatewayHubImpl_GATEWAYHUB_0_DOES_NOT_CONTAIN_A_GATEWAY_WITH_ID_1 = new StringIdImpl(2674, "GatewayHub  {0}  does not contain a Gateway with id= {1}");
   public static final StringId GatewayImpl_A_GATEWAYS_CONFIGURATION_CANNOT_BE_CHANGED_ONCE_IT_IS_RUNNING = new StringIdImpl(2675, "A Gateway''s configuration cannot be changed once it is running.");
@@ -2400,20 +2384,7 @@ class ParentLocalizedStrings {
   public static final StringId DistributedRegion_OPERATION_DISTRIBUTION_MAY_HAVE_FAILED_TO_NOTIFY_THESE_REQUIRED_ROLES_0 = new StringIdImpl( 3396, "Operation distribution may have failed to notify these required roles: {0}");
   public static final StringId DistributedRegion_OPERATION_DISTRIBUTION_WAS_NOT_DONE_TO_THESE_REQUIRED_ROLES_0 = new StringIdImpl( 3397, "Operation distribution was not done to these required roles: {0}");
   public static final StringId DistributedCacheOperation_THE_CACHE_HAS_BEEN_CLOSED = new StringIdImpl(3398, "The cache has been closed");
-  public static final StringId BridgeLoader_SOCKET_CLOSED_ON_SERVER = new StringIdImpl( 3399, "socket closed on server");
-  public static final StringId BridgeLoader_SOCKET_TIMED_OUT_ON_CLIENT = new StringIdImpl( 3400, "socket timed out on client");
-  public static final StringId BridgeLoader_SERVER_DEATH = new StringIdImpl( 3401, "server death");
-  public static final StringId BridgeLoader_BRIDGELOADER_CONNECTED_TO_0 = new StringIdImpl( 3402, "BridgeLoader connected to {0}");
-  public static final StringId BridgeLoader_NO_ACTIVE_SERVERS_WERE_FOUND = new StringIdImpl( 3403, "No active servers were found.");
-  public static final StringId BridgeLoader_NO_AVAILABLE_CONNECTION_WAS_FOUND_BUT_THE_FOLLOWING_ACTIVE_SERVERS_EXIST_0 = new StringIdImpl( 3404, "No available connection was found, but the following active server(s) exist: {0}");
-  public static final StringId BridgeWriter_THE_BRIDGEWRITER_HAS_BEEN_CLOSED_WHILE_ATTEMPTING_TO_LOAD_REGION_0_KEY_1 = new StringIdImpl( 3405, "The BridgeWriter has been closed while attempting to load: region={0} key={1}");
-  public static final StringId BridgeWriter_NO_ACTIVE_SERVERS_WERE_FOUND_WHILE_ATTEMPTING_TO_LOAD_REGION_0_KEY_1 = new StringIdImpl( 3406, "No active servers were found while attempting to load: region={0} key={1}");
-  public static final StringId BridgeWriter_NO_AVAILABLE_CONNECTION_WAS_FOUND_WHILE_ATTEMPTING_TO_LOAD_REGION_0_KEY_1_BUT_THE_FOLLOWING_ACTIVE_SERVERS_EXIST_2 = new StringIdImpl( 3407, "No available Connection was found while attempting to load: region={0} key={1}, but the following active servers exist: {2}");
-  public static final StringId BridgeWriter_SOCKET_TIMED_OUT_ON_CLIENT = new StringIdImpl( 3408, "socket timed out on client");
-  public static final StringId BridgeWriter_SOCKET_CLOSED_ON_SERVER = new StringIdImpl( 3409, "socket closed on server");
-  public static final StringId BridgeWriter_SERVER_DEATH = new StringIdImpl( 3410, "server death");
-  public static final StringId BridgeWriter_SERVER_UNREACHABLE_COULD_NOT_CONNECT_AFTER_0_ATTEMPTS = new StringIdImpl( 3411, "Server unreachable: could not connect after {0} attempts");
-  public static final StringId BridgeWriter_BRIDGEWRITER_CONNECTED_TO_0 = new StringIdImpl( 3412, "BridgeWriter connected to {0}");
+  // okay to reuse 3399..3412
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/CacheServerBridge.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/CacheServerBridge.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/CacheServerBridge.java
index 012b8ec..130482e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/CacheServerBridge.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/CacheServerBridge.java
@@ -35,14 +35,13 @@ import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.server.ServerLoad;
 import com.gemstone.gemfire.cache.server.ServerLoadProbe;
 import com.gemstone.gemfire.cache.server.internal.ServerMetricsImpl;
-import com.gemstone.gemfire.cache.util.BridgeMembershipListener;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.admin.ClientHealthMonitoringRegion;
 import com.gemstone.gemfire.internal.admin.remote.ClientHealthStats;
-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.ha.HARegionQueue;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientNotifier;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy;
@@ -60,6 +59,7 @@ import com.gemstone.gemfire.management.internal.beans.stats.StatsAverageLatency;
 import com.gemstone.gemfire.management.internal.beans.stats.StatsKey;
 import com.gemstone.gemfire.management.internal.beans.stats.StatsRate;
 import com.gemstone.gemfire.management.internal.cli.CliUtil;
+import com.gemstone.gemfire.management.membership.ClientMembershipListener;
 
 /**
  * Represents the GemFire CacheServer . Provides data and notifications about
@@ -86,7 +86,7 @@ public class CacheServerBridge extends ServerBridge{
   
   private MemberMBeanBridge memberMBeanBridge;
 
-  private BridgeMembershipListener membershipListener;
+  private ClientMembershipListener membershipListener;
   
   public static ThreadLocal<Version> clientVersion = new ThreadLocal<Version>();
 
@@ -406,7 +406,7 @@ public class CacheServerBridge extends ServerBridge{
       return null;      
     }
        
-    BridgeServerImpl server = (BridgeServerImpl)cache.getCacheServers().iterator().next();
+    CacheServerImpl server = (CacheServerImpl)cache.getCacheServers().iterator().next();
     
     if(server == null){
       return null;
@@ -663,16 +663,16 @@ public class CacheServerBridge extends ServerBridge{
   }
 
   public int getNumSubscriptions() {
-    Map clientProxyMembershipIDMap = InternalBridgeMembership.getClientQueueSizes();
+    Map clientProxyMembershipIDMap = InternalClientMembership.getClientQueueSizes();
     return clientProxyMembershipIDMap.keySet().size();
   }
 
-  public void setBridgeMembershipListener(
-      BridgeMembershipListener membershipListener) {
+  public void setClientMembershipListener(
+      ClientMembershipListener membershipListener) {
     this.membershipListener = membershipListener;
   }
   
-  public BridgeMembershipListener getBridgeMembershipListener() {
+  public ClientMembershipListener getClientMembershipListener() {
     return this.membershipListener;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/GatewayReceiverMBeanBridge.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/GatewayReceiverMBeanBridge.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/GatewayReceiverMBeanBridge.java
index 6358752..935d5d1 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/GatewayReceiverMBeanBridge.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/GatewayReceiverMBeanBridge.java
@@ -15,7 +15,7 @@ import java.util.Set;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.wan.GatewayReceiver;
 import com.gemstone.gemfire.cache.wan.GatewayTransportFilter;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.cache.wan.GatewayReceiverStats;
@@ -187,7 +187,7 @@ public class GatewayReceiverMBeanBridge extends ServerBridge{
 
   public String[] getConnectedGatewaySenders() {
     Set<String> uniqueIds = null;
-    AcceptorImpl acceptor = ((BridgeServerImpl)rcv.getServer()).getAcceptor();
+    AcceptorImpl acceptor = ((CacheServerImpl)rcv.getServer()).getAcceptor();
     Set<ServerConnection> serverConnections = acceptor.getAllServerConnections();
     if(serverConnections !=null && serverConnections.size() >0){
       uniqueIds = new HashSet<String>();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java
index 3d6331d..d22480b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ManagementAdapter.java
@@ -31,10 +31,6 @@ import com.gemstone.gemfire.cache.DiskStore;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.cache.util.BridgeMembership;
-import com.gemstone.gemfire.cache.util.BridgeMembershipEvent;
-import com.gemstone.gemfire.cache.util.BridgeMembershipListener;
-import com.gemstone.gemfire.cache.util.BridgeMembershipListenerAdapter;
 import com.gemstone.gemfire.cache.wan.GatewayReceiver;
 import com.gemstone.gemfire.cache.wan.GatewaySender;
 import com.gemstone.gemfire.distributed.Locator;
@@ -66,6 +62,10 @@ import com.gemstone.gemfire.management.internal.FederationComponent;
 import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
 import com.gemstone.gemfire.management.internal.ManagementConstants;
 import com.gemstone.gemfire.management.internal.SystemManagementService;
+import com.gemstone.gemfire.management.membership.ClientMembership;
+import com.gemstone.gemfire.management.membership.ClientMembershipEvent;
+import com.gemstone.gemfire.management.membership.ClientMembershipListener;
+import com.gemstone.gemfire.management.membership.ClientMembershipListenerAdapter;
 import com.gemstone.gemfire.pdx.internal.PeerTypeRegistration;
 
 /**
@@ -691,11 +691,11 @@ public class ManagementAdapter {
     ObjectName changedMBeanName = service.registerInternalMBean(
         (CacheServerMXBean) cacheServerMBean, cacheServerMBeanName);
     
-    BridgeMembershipListener managementBridgeListener = new CacheServerMembershipListenerAdapter(cacheServerMBean,
+    ClientMembershipListener managementClientListener = new CacheServerMembershipListenerAdapter(cacheServerMBean,
         memberLevelNotifEmitter, changedMBeanName);
-    BridgeMembership.registerBridgeMembershipListener(managementBridgeListener);
+    ClientMembership.registerClientMembershipListener(managementClientListener);
     
-    cacheServerBridge.setBridgeMembershipListener(managementBridgeListener);
+    cacheServerBridge.setClientMembershipListener(managementClientListener);
     
     service.federate(changedMBeanName, CacheServerMXBean.class, true);
     
@@ -725,11 +725,11 @@ public class ManagementAdapter {
     CacheServerMBean mbean = (CacheServerMBean) service
         .getLocalCacheServerMXBean(server.getPort());
     
-    BridgeMembershipListener listener = mbean.getBridge()
-        .getBridgeMembershipListener();
+    ClientMembershipListener listener = mbean.getBridge()
+        .getClientMembershipListener();
     
     if(listener != null){
-      BridgeMembership.unregisterBridgeMembershipListener(listener);
+      ClientMembership.unregisterClientMembershipListener(listener);
     }
    
 
@@ -824,11 +824,11 @@ public class ManagementAdapter {
             .getLocalCacheServerMXBean(server.getPort());
 
         if (mbean != null) {
-          BridgeMembershipListener listener = mbean.getBridge()
-            .getBridgeMembershipListener();
+          ClientMembershipListener listener = mbean.getBridge()
+            .getClientMembershipListener();
 
           if (listener != null) {
-            BridgeMembership.unregisterBridgeMembershipListener(listener);
+            ClientMembership.unregisterClientMembershipListener(listener);
           }
         }
 
@@ -1057,12 +1057,12 @@ public class ManagementAdapter {
 
   
   /**
-   * Private class which acts as a BridgeMembershipListener to propagate client
+   * Private class which acts as a ClientMembershipListener to propagate client
    * joined/left notifications
    */
 
   private static class CacheServerMembershipListenerAdapter extends
-      BridgeMembershipListenerAdapter {
+      ClientMembershipListenerAdapter {
     
     private NotificationBroadcasterSupport serverLevelNotifEmitter;
     private NotificationBroadcasterSupport memberLevelNotifEmitter;
@@ -1078,9 +1078,9 @@ public class ManagementAdapter {
 
     /**
      * Invoked when a client has connected to this process or when this process
-     * has connected to a BridgeServer.
+     * has connected to a CacheServer.
      */
-    public void memberJoined(BridgeMembershipEvent event) {
+    public void memberJoined(ClientMembershipEvent event) {
       Notification notification = new Notification(JMXNotificationType.CLIENT_JOINED, serverSource, SequenceNumber
           .next(), System.currentTimeMillis(), ManagementConstants.CLIENT_JOINED_PREFIX + event.getMemberId());
       serverLevelNotifEmitter.sendNotification(notification);
@@ -1090,9 +1090,9 @@ public class ManagementAdapter {
 
     /**
      * Invoked when a client has gracefully disconnected from this process or
-     * when this process has gracefully disconnected from a BridgeServer.
+     * when this process has gracefully disconnected from a CacheServer.
      */
-    public void memberLeft(BridgeMembershipEvent event) {
+    public void memberLeft(ClientMembershipEvent event) {
       Notification notification = new Notification(JMXNotificationType.CLIENT_LEFT, serverSource, SequenceNumber
           .next(), System.currentTimeMillis(), ManagementConstants.CLIENT_LEFT_PREFIX + event.getMemberId());
       serverLevelNotifEmitter.sendNotification(notification);
@@ -1101,9 +1101,9 @@ public class ManagementAdapter {
 
     /**
      * Invoked when a client has unexpectedly disconnected from this process or
-     * when this process has unexpectedly disconnected from a BridgeServer.
+     * when this process has unexpectedly disconnected from a CacheServer.
      */
-    public void memberCrashed(BridgeMembershipEvent event) {
+    public void memberCrashed(ClientMembershipEvent event) {
       Notification notification = new Notification(JMXNotificationType.CLIENT_CRASHED, serverSource, SequenceNumber
           .next(), System.currentTimeMillis(), ManagementConstants.CLIENT_CRASHED_PREFIX + event.getMemberId());
       serverLevelNotifEmitter.sendNotification(notification);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ServerBridge.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ServerBridge.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ServerBridge.java
index 81cc34c..4fbc245 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ServerBridge.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/beans/ServerBridge.java
@@ -8,7 +8,7 @@
 package com.gemstone.gemfire.management.internal.beans;
 
 import com.gemstone.gemfire.cache.server.CacheServer;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
 import com.gemstone.gemfire.management.internal.ManagementStrings;
@@ -38,7 +38,7 @@ public class ServerBridge {
   public ServerBridge(CacheServer cacheServer){
     this.monitor = new MBeanStatsMonitor(ManagementStrings.SERVER_MONITOR
         .toLocalizedString());
-    this.acceptor =  ((BridgeServerImpl) cacheServer).getAcceptor();
+    this.acceptor =  ((CacheServerImpl) cacheServer).getAcceptor();
     initializeStats();
     startMonitor();
   }
@@ -48,7 +48,7 @@ public class ServerBridge {
   }
   
   protected void addServer(CacheServer cacheServer){
-    this.acceptor =  ((BridgeServerImpl) cacheServer).getAcceptor();
+    this.acceptor =  ((CacheServerImpl) cacheServer).getAcceptor();
     startMonitor();
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/ContunuousQueryFunction.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/ContunuousQueryFunction.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/ContunuousQueryFunction.java
index acdbc0c..7139263 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/ContunuousQueryFunction.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/ContunuousQueryFunction.java
@@ -16,7 +16,7 @@ import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionContext;
 import com.gemstone.gemfire.internal.InternalEntity;
-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.sockets.AcceptorImpl;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientNotifier;
@@ -42,7 +42,7 @@ public class ContunuousQueryFunction implements Function, InternalEntity {
       String clientID = (String) context.getArguments();      
       GemFireCacheImpl cache = (GemFireCacheImpl)CacheFactory.getAnyInstance();      
       if (cache.getCacheServers().size() > 0) {       
-        BridgeServerImpl server = (BridgeServerImpl)cache.getCacheServers().iterator().next();        
+        CacheServerImpl server = (CacheServerImpl)cache.getCacheServers().iterator().next();        
         if(server != null){          
           AcceptorImpl  acceptorImpl  = server.getAcceptor(); 
           if(acceptorImpl != null){