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

[01/38] incubator-geode git commit: GEODE-324: defer deserialization of old value in PR put

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-77 3e1a9f69b -> c4b14b724


GEODE-324: defer deserialization of old value in PR put

When a PR put returns the old value our internal messages were deserializing
the values before we get back to the user thread that initiated the operation.
If a thread local class loader is used for the class we need to load during
deserialization then the early deserialization will fail.

Now the deserialization is delayed until the serialized old value gets through
our messaging layer and back to the thread that performed the pr put.

Added junit tests that verify that both PutReplyMessage classes no longer
deserialize their old value.


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

Branch: refs/heads/feature/GEODE-77
Commit: d8570c980d10827a82fc94aaeaaddb47d7c6d242
Parents: 27eceb1
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Sep 10 11:43:26 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Sep 30 11:05:42 2015 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/DataSerializer.java    |  11 +-
 .../internal/InternalDataSerializer.java        |  33 ++--
 .../cache/DistributedCacheOperation.java        |   8 +-
 .../internal/cache/RemotePutMessage.java        |  56 ++++---
 .../internal/cache/partitioned/PutMessage.java  |  21 ++-
 .../offheap/OffHeapCachedDeserializable.java    |  20 +++
 .../offheap/SimpleMemoryAllocatorImpl.java      |  36 ++++
 .../gemfire/internal/offheap/StoredObject.java  |  26 ++-
 .../cache/OldValueImporterTestBase.java         | 165 +++++++++++++++++++
 .../cache/RemotePutReplyMessageJUnitTest.java   |  37 +++++
 .../PutPutReplyMessageJUnitTest.java            |  38 +++++
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java | 101 ++++++++++++
 .../gemfire/pdx/ByteSourceJUnitTest.java        |   4 +-
 .../sanctionedDataSerializables.txt             |   4 +-
 14 files changed, 511 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java
index 8f977d8..685f546 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java
@@ -1475,9 +1475,16 @@ public abstract class DataSerializer {
     throws IOException {
     Object object = obj;
     if (obj instanceof CachedDeserializable) {
-     // if( ( !(obj instanceof ByteSource) || ((StoredObject) obj).isSerialized())) {
+      if (obj instanceof StoredObject) {
+        StoredObject so = (StoredObject)obj;
+        if (logger.isTraceEnabled(LogMarker.SERIALIZER)) {
+          logger.trace(LogMarker.SERIALIZER, "writeObjectAsByteArray StoredObject");
+        }
+        so.sendAsByteArray(out);
+        return;
+      } else {
         object = ((CachedDeserializable) obj).getSerializedValue();
-     // }
+      }
     }
     if (logger.isTraceEnabled(LogMarker.SERIALIZER)) {
       if (object == null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
index 87ab28d..09f9280 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
@@ -1361,16 +1361,30 @@ public abstract class InternalDataSerializer extends DataSerializer implements D
 
   ///////////////// START DataSerializer Implementation Methods ///////////
 
+  // Writes just the header of a DataSerializableFixedID to out.
+  public static final void writeDSFIDHeader(int dsfid, DataOutput out) throws IOException {
+    if (dsfid == DataSerializableFixedID.ILLEGAL) {
+      throw new IllegalStateException(LocalizedStrings.InternalDataSerializer_ATTEMPTED_TO_SERIALIZE_ILLEGAL_DSFID.toLocalizedString());
+    }
+   if (dsfid <= Byte.MAX_VALUE && dsfid >= Byte.MIN_VALUE) {
+      out.writeByte(DS_FIXED_ID_BYTE);
+      out.writeByte(dsfid);
+    } else if (dsfid <= Short.MAX_VALUE && dsfid >= Short.MIN_VALUE) {
+      out.writeByte(DS_FIXED_ID_SHORT);
+      out.writeShort(dsfid);
+    } else {
+      out.writeByte(DS_FIXED_ID_INT);
+      out.writeInt(dsfid);
+    }
+  }
+  
   public static final void writeDSFID(DataSerializableFixedID o, DataOutput out)
     throws IOException
   {
     int dsfid = o.getDSFID();
-    if (dsfid == DataSerializableFixedID.ILLEGAL) {
-      throw new IllegalStateException(LocalizedStrings.InternalDataSerializer_ATTEMPTED_TO_SERIALIZE_ILLEGAL_DSFID.toLocalizedString());
-    }
     if (dsfidToClassMap != null && logger.isTraceEnabled(LogMarker.DEBUG_DSFID)) {
       logger.trace(LogMarker.DEBUG_DSFID, "writeDSFID {} class={}", dsfid, o.getClass());
-      if (dsfid != DataSerializableFixedID.NO_FIXED_ID) {
+      if (dsfid != DataSerializableFixedID.NO_FIXED_ID && dsfid != DataSerializableFixedID.ILLEGAL) {
         // consistency check to make sure that the same DSFID is not used
         // for two different classes
         String newClassName = o.getClass().getName();
@@ -1380,18 +1394,11 @@ public abstract class InternalDataSerializer extends DataSerializer implements D
         }
       }
     }
-    if (dsfid <= Byte.MAX_VALUE && dsfid >= Byte.MIN_VALUE) {
-      out.writeByte(DS_FIXED_ID_BYTE);
-      out.writeByte(dsfid);
-    } else if (dsfid <= Short.MAX_VALUE && dsfid >= Short.MIN_VALUE) {
-      out.writeByte(DS_FIXED_ID_SHORT);
-      out.writeShort(dsfid);
-    } else if (dsfid == DataSerializableFixedID.NO_FIXED_ID) {
+    if (dsfid == DataSerializableFixedID.NO_FIXED_ID) {
       out.writeByte(DS_NO_FIXED_ID);
       DataSerializer.writeClass(o.getClass(), out);
     } else {
-      out.writeByte(DS_FIXED_ID_INT);
-      out.writeInt(dsfid);
+      writeDSFIDHeader(dsfid, out);
     }
     try {
       invokeToData(o, out);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
index 692595c..fc9046c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
@@ -65,7 +65,7 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
-import com.gemstone.gemfire.internal.offheap.OffHeapReference;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
 
@@ -117,9 +117,9 @@ public abstract class DistributedCacheOperation {
         DataSerializer.writeObject(vObj, out);
       } else if (deserializationPolicy == DESERIALIZATION_POLICY_NONE) {
         // We only have NONE with a vObj when vObj is off-heap and not serialized.
-        OffHeapReference ohref = (OffHeapReference) vObj;
-        assert !ohref.isSerialized();
-        DataSerializer.writeByteArray(ohref.getValueAsHeapByteArray(), out);
+        StoredObject so = (StoredObject) vObj;
+        assert !so.isSerialized();
+        so.sendAsByteArray(out);
       } else { // LAZY
         // TODO OFFHEAP MERGE: cache the oldValue that is serialized here
         // into the event

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
index 3f5efeb..a879adf 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
@@ -54,7 +54,6 @@ import com.gemstone.gemfire.internal.util.Breadcrumbs;
 
 import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_OLD_VALUE;
 import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_NEW_VALUE;
-
 import static com.gemstone.gemfire.internal.cache.DistributedCacheOperation.VALUE_IS_BYTES;
 import static com.gemstone.gemfire.internal.cache.DistributedCacheOperation.VALUE_IS_SERIALIZED_OBJECT;
 import static com.gemstone.gemfire.internal.cache.DistributedCacheOperation.VALUE_IS_OBJECT;
@@ -880,7 +879,7 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
      * or null if not set.
      */
     @Unretained(ENTRY_EVENT_OLD_VALUE)
-    Object oldValue;
+    private Object oldValue;
     
     /**
      * version tag for concurrency control
@@ -898,7 +897,8 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
     public PutReplyMessage() {
     }
 
-    private PutReplyMessage(int processorId,
+    // unit tests may call this constructor
+    PutReplyMessage(int processorId,
                             boolean result,
                             Operation op,
                             ReplyException ex,
@@ -965,13 +965,15 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
       dm.getStats().incReplyMessageTime(NanoTimer.getTime()-startTime);
     }
 
-    /** Return oldValue in deserialized form */
+    /** Return oldValue as a byte[] or as a CachedDeserializable.
+     * This method used to deserialize a CachedDeserializable but that is too soon.
+     * This method is called during message processing. The deserialization needs
+     * to be deferred until we get back to the application thread which happens
+     * for this oldValue when they call EntryEventImpl.getOldValue.
+     */
     public Object getOldValue() {
       // oldValue field is in serialized form, either a CachedDeserializable,
       // a byte[], or null if not set
-      if (this.oldValue instanceof CachedDeserializable) {
-        return ((CachedDeserializable)this.oldValue).getDeserializedValue(null, null);
-      }
       return this.oldValue;
     }
 
@@ -994,7 +996,29 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
       }
     }
 
-    @Override
+    public static void oldValueToData(DataOutput out, Object ov, boolean ovIsSerialized) throws IOException {
+      if (ovIsSerialized && ov != null) {
+        byte[] oldValueBytes;
+        if (ov instanceof byte[]) {
+          oldValueBytes = (byte[]) ov;
+          DataSerializer.writeObject(new VMCachedDeserializable(oldValueBytes), out);
+        } else if (ov instanceof CachedDeserializable) {
+          if (ov instanceof StoredObject) {
+            ((StoredObject) ov).sendAsCachedDeserializable(out);
+          } else {
+            DataSerializer.writeObject(ov, out);
+          }
+        } else {
+          oldValueBytes = EntryEventImpl.serialize(ov);
+          DataSerializer.writeObject(new VMCachedDeserializable(oldValueBytes), out);
+        }
+      } else {
+        DataSerializer.writeObject(ov, out);
+      }
+      
+    }
+    
+     @Override
     public void toData(DataOutput out) throws IOException {
       super.toData(out);
       byte flags = 0;
@@ -1003,12 +1027,7 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
       if (this.versionTag instanceof DiskVersionTag) flags |= FLAG_PERSISTENT;
       out.writeByte(flags);
       out.writeByte(this.op.ordinal);
-      if (this.oldValueIsSerialized) {
-        byte[] oldValueBytes = (byte[]) this.oldValue;
-        out.write(oldValueBytes);
-      } else {
-        DataSerializer.writeObject(this.oldValue, out);
-      }
+      oldValueToData(out, getOldValue(), this.oldValueIsSerialized);
       if (this.versionTag != null) {
         InternalDataSerializer.invokeToData(this.versionTag, out);
       }
@@ -1045,18 +1064,13 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
 
     @Override
     public void importOldObject(@Unretained(ENTRY_EVENT_OLD_VALUE) Object ov, boolean isSerialized) {
-      // isSerialized does not matter.
-      // toData will just call writeObject
-      // and fromData will just call readObject
+      this.oldValueIsSerialized = isSerialized;
       this.oldValue = ov;
     }
 
     @Override
     public void importOldBytes(byte[] ov, boolean isSerialized) {
-      if (isSerialized) {
-        this.oldValueIsSerialized = true;
-      }
-      this.oldValue = ov;
+      importOldObject(ov, isSerialized);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
index 1f7b08e..e47ce04 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
@@ -51,6 +51,8 @@ import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.PrimaryBucketException;
+import com.gemstone.gemfire.internal.cache.RemotePutMessage;
+import com.gemstone.gemfire.internal.cache.VMCachedDeserializable;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -1036,6 +1038,14 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
 
     VersionTag versionTag;
 
+    /**
+     * Set to true by the import methods if the oldValue
+     * is already serialized. In that case toData
+     * should just copy the bytes to the stream.
+     * In either case fromData just calls readObject.
+     */
+    private transient boolean oldValueIsSerialized;
+
     @Override
     public boolean getInlineProcess() {
       return true;
@@ -1047,7 +1057,8 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     public PutReplyMessage() {
     }
     
-    private PutReplyMessage(int processorId,
+    // package access for unit test
+    PutReplyMessage(int processorId,
                             boolean result,
                             Operation op,
                             ReplyException ex,
@@ -1139,12 +1150,13 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
       this.versionTag = (VersionTag)DataSerializer.readObject(in);
     }
 
-    @Override
+   @Override
     public void toData(DataOutput out) throws IOException {
       super.toData(out);
       out.writeBoolean(this.result);
       out.writeByte(this.op.ordinal);
-      DataSerializer.writeObject(this.oldValue, out);
+      Object ov = getOldValue();
+      RemotePutMessage.PutReplyMessage.oldValueToData(out, getOldValue(), this.oldValueIsSerialized);
       DataSerializer.writeObject(this.versionTag, out);
     }
 
@@ -1180,11 +1192,12 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     @Override
     public void importOldObject(@Unretained(ENTRY_EVENT_OLD_VALUE) Object ov, boolean isSerialized) {
       this.oldValue = ov;
+      this.oldValueIsSerialized = isSerialized;
     }
 
     @Override
     public void importOldBytes(byte[] ov, boolean isSerialized) {
-      this.oldValue = ov;
+      importOldObject(ov, isSerialized);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
index 764a051..4d22171 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapCachedDeserializable.java
@@ -7,6 +7,8 @@ import java.util.Arrays;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.internal.DSCODE;
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
 import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
 import com.gemstone.gemfire.internal.cache.EntryBits;
@@ -106,6 +108,24 @@ public abstract class OffHeapCachedDeserializable implements MemoryChunkWithRefC
       DataSerializer.writeObject(objToSend, out);
     }
   }
+  @Override
+  public void sendAsByteArray(DataOutput out) throws IOException {
+    byte[] bytes;
+    if (isSerialized()) {
+      bytes = getSerializedValue();
+    } else {
+      bytes = (byte[]) getDeserializedForReading();
+    }
+    DataSerializer.writeByteArray(bytes, out);
+  }
+  @Override
+  public void sendAsCachedDeserializable(DataOutput out) throws IOException {
+    if (!isSerialized()) {
+      throw new IllegalStateException("sendAsCachedDeserializable can only be called on serialized StoredObjects");
+    }
+    InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
+    sendAsByteArray(out);
+  }
   public boolean checkDataEquals(@Unretained OffHeapCachedDeserializable other) {
     if (this == other) {
       return true;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
index 7cf1656..4652620 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
@@ -42,6 +42,7 @@ import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.DSCODE;
+import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.cache.BucketRegion;
@@ -527,6 +528,27 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
     }
 
     @Override
+    public void sendAsByteArray(DataOutput out) throws IOException {
+      byte[] bytes;
+      if (isSerialized()) {
+        bytes = getSerializedValue();
+      } else {
+        bytes = (byte[]) getDeserializedForReading();
+      }
+      DataSerializer.writeByteArray(bytes, out);
+      
+    }
+    
+    @Override
+    public void sendAsCachedDeserializable(DataOutput out) throws IOException {
+      if (!isSerialized()) {
+        throw new IllegalStateException("sendAsCachedDeserializable can only be called on serialized StoredObjects");
+      }
+      InternalDataSerializer.writeDSFIDHeader(DataSerializableFixedID.VM_CACHED_DESERIALIZABLE, out);
+      sendAsByteArray(out);
+    }
+
+    @Override
     public boolean isSerialized() {
       return OffHeapRegionEntryHelper.isSerialized(this.address);
     }
@@ -1816,6 +1838,20 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
       }
       super.sendTo(out);
     }
+    
+    @Override
+    public void sendAsByteArray(DataOutput out) throws IOException {
+      if (!isCompressed() && out instanceof HeapDataOutputStream) {
+        ByteBuffer bb = createDirectByteBuffer();
+        if (bb != null) {
+          HeapDataOutputStream hdos = (HeapDataOutputStream) out;
+          InternalDataSerializer.writeArrayLength(bb.remaining(), hdos);
+          hdos.write(bb);
+          return;
+        }
+      }
+      super.sendAsByteArray(out);
+    }
        
     private static volatile Class dbbClass = null;
     private static volatile Constructor dbbCtor = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
index 8f179ae..90b65e9 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/StoredObject.java
@@ -1,5 +1,8 @@
 package com.gemstone.gemfire.internal.offheap;
 
+import java.io.DataOutput;
+import java.io.IOException;
+
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
 
 /**
@@ -11,5 +14,26 @@ import com.gemstone.gemfire.internal.cache.CachedDeserializable;
  * @since 9.0
  */
 public interface StoredObject extends OffHeapReference, CachedDeserializable {
-
+  /**
+   * Take all the bytes in the object and write them to the data output as a byte array.
+   * If the StoredObject is not serialized then its raw byte array is sent.
+   * But if it is serialized then the serialized byte array is sent.
+   * The corresponding de-serialization will need to call readByteArray.
+   * 
+   * @param out
+   *          the data output to send this object to
+   * @throws IOException
+   */
+  void sendAsByteArray(DataOutput out) throws IOException;
+  /**
+   * Take all the bytes in the object and write them to the data output as a byte array.
+   * If the StoredObject is not serialized then an exception will be thrown.
+   * The corresponding deserialization will need to call readObject and will get an
+   * instance of VMCachedDeserializable.
+   * 
+   * @param out
+   *          the data output to send this object to
+   * @throws IOException
+   */
+  void sendAsCachedDeserializable(DataOutput out) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
new file mode 100644
index 0000000..a2f6ef1
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/OldValueImporterTestBase.java
@@ -0,0 +1,165 @@
+package com.gemstone.gemfire.internal.cache;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl.OldValueImporter;
+import com.gemstone.gemfire.internal.offheap.NullOffHeapMemoryStats;
+import com.gemstone.gemfire.internal.offheap.NullOutOfOffHeapMemoryListener;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.DataAsAddress;
+import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
+import com.gemstone.gemfire.internal.util.BlobHelper;
+
+public abstract class OldValueImporterTestBase {
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+  
+  protected abstract OldValueImporter createImporter();
+  protected abstract Object getOldValueFromImporter(OldValueImporter ovi);
+  protected abstract void toData(OldValueImporter ovi, HeapDataOutputStream hdos) throws IOException;
+  protected abstract void fromData(OldValueImporter ovi, byte[] bytes) throws IOException, ClassNotFoundException;
+
+  @Test
+  public void testValueSerialization() throws IOException, ClassNotFoundException {
+    byte[] bytes = new byte[1024];
+    HeapDataOutputStream hdos = new HeapDataOutputStream(bytes);
+    OldValueImporter imsg = createImporter();
+
+    // null byte array value
+    {
+      OldValueImporter omsg = createImporter();
+      omsg.importOldBytes(null, false);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertEquals(null, getOldValueFromImporter(imsg));
+    }
+    
+    // null object value
+    {
+      OldValueImporter omsg = createImporter();
+      omsg.importOldObject(null, true);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertEquals(null, getOldValueFromImporter(imsg));
+    }
+    
+    // simple byte array
+    {
+      byte[] baValue = new byte[] {1,2,3,4,5,6,7,8,9};
+      OldValueImporter omsg = createImporter();
+      omsg.importOldBytes(baValue, false);
+      hdos = new HeapDataOutputStream(bytes);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertArrayEquals(baValue, (byte[])getOldValueFromImporter(imsg));
+    }
+    
+    // String in serialized form
+    {
+      String stringValue = "1,2,3,4,5,6,7,8,9";
+      byte[] stringValueBlob = EntryEventImpl.serialize(stringValue);
+      OldValueImporter omsg = createImporter();
+      omsg.importOldBytes(stringValueBlob, true);
+      hdos = new HeapDataOutputStream(bytes);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertArrayEquals(stringValueBlob, ((VMCachedDeserializable)getOldValueFromImporter(imsg)).getSerializedValue());
+    }
+    
+    // String in object form
+    {
+      String stringValue = "1,2,3,4,5,6,7,8,9";
+      byte[] stringValueBlob = EntryEventImpl.serialize(stringValue);
+      OldValueImporter omsg = createImporter();
+      omsg.importOldObject(stringValue, true);
+      hdos = new HeapDataOutputStream(bytes);
+      toData(omsg, hdos);
+      fromData(imsg, bytes);
+      assertArrayEquals(stringValueBlob, ((VMCachedDeserializable)getOldValueFromImporter(imsg)).getSerializedValue());
+    }
+    
+    // off-heap DataAsAddress byte array
+    {
+      SimpleMemoryAllocatorImpl sma =
+          SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+      try {
+        byte[] baValue = new byte[] {1,2};
+        DataAsAddress baValueSO = (DataAsAddress) sma.allocateAndInitialize(baValue, false, false, null);
+        OldValueImporter omsg = createImporter();
+        omsg.importOldObject(baValueSO, false);
+        hdos = new HeapDataOutputStream(bytes);
+        toData(omsg, hdos);
+        fromData(imsg, bytes);
+        assertArrayEquals(baValue, (byte[])getOldValueFromImporter(imsg));
+      } finally {
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+    // off-heap Chunk byte array
+    {
+      SimpleMemoryAllocatorImpl sma =
+          SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+      try {
+        byte[] baValue = new byte[] {1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17};
+        Chunk baValueSO = (Chunk) sma.allocateAndInitialize(baValue, false, false, null);
+        OldValueImporter omsg = createImporter();
+        omsg.importOldObject(baValueSO, false);
+        hdos = new HeapDataOutputStream(bytes);
+        toData(omsg, hdos);
+        fromData(imsg, bytes);
+        assertArrayEquals(baValue, (byte[])getOldValueFromImporter(imsg));
+      } finally {
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+    // off-heap DataAsAddress String
+    {
+      SimpleMemoryAllocatorImpl sma =
+          SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+      try {
+        String baValue = "12";
+        byte[] baValueBlob = BlobHelper.serializeToBlob(baValue);
+        DataAsAddress baValueSO = (DataAsAddress) sma.allocateAndInitialize(baValueBlob, true, false, null);
+        OldValueImporter omsg = createImporter();
+        omsg.importOldObject(baValueSO, true);
+        hdos = new HeapDataOutputStream(bytes);
+        toData(omsg, hdos);
+        fromData(imsg, bytes);
+        assertArrayEquals(baValueBlob, ((VMCachedDeserializable)getOldValueFromImporter(imsg)).getSerializedValue());
+      } finally {
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+    // off-heap Chunk String
+    {
+      SimpleMemoryAllocatorImpl sma =
+          SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+      try {
+        String baValue = "12345678";
+        byte[] baValueBlob = BlobHelper.serializeToBlob(baValue);
+        Chunk baValueSO = (Chunk) sma.allocateAndInitialize(baValueBlob, true, false, null);
+        OldValueImporter omsg = createImporter();
+        omsg.importOldObject(baValueSO, true);
+        hdos = new HeapDataOutputStream(bytes);
+        toData(omsg, hdos);
+        fromData(imsg, bytes);
+        assertArrayEquals(baValueBlob, ((VMCachedDeserializable)getOldValueFromImporter(imsg)).getSerializedValue());
+      } finally {
+        SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemotePutReplyMessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemotePutReplyMessageJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemotePutReplyMessageJUnitTest.java
new file mode 100644
index 0000000..4c1ae33
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemotePutReplyMessageJUnitTest.java
@@ -0,0 +1,37 @@
+package com.gemstone.gemfire.internal.cache;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl.OldValueImporter;
+import com.gemstone.gemfire.internal.cache.RemotePutMessage.PutReplyMessage;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class RemotePutReplyMessageJUnitTest extends OldValueImporterTestBase {
+
+  @Override
+  protected OldValueImporter createImporter() {
+    return new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null);
+  }
+
+  @Override
+  protected Object getOldValueFromImporter(OldValueImporter ovi) {
+    return ((PutReplyMessage)ovi).getOldValue();
+  }
+
+  @Override
+  protected void toData(OldValueImporter ovi, HeapDataOutputStream hdos) throws IOException {
+    ((PutReplyMessage)ovi).toData(hdos);
+  }
+
+  @Override
+  protected void fromData(OldValueImporter ovi, byte[] bytes) throws IOException, ClassNotFoundException {
+    ((PutReplyMessage)ovi).fromData(new DataInputStream(new ByteArrayInputStream(bytes)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PutPutReplyMessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PutPutReplyMessageJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PutPutReplyMessageJUnitTest.java
new file mode 100644
index 0000000..5e89aa8
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/PutPutReplyMessageJUnitTest.java
@@ -0,0 +1,38 @@
+package com.gemstone.gemfire.internal.cache.partitioned;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl.OldValueImporter;
+import com.gemstone.gemfire.internal.cache.OldValueImporterTestBase;
+import com.gemstone.gemfire.internal.cache.partitioned.PutMessage.PutReplyMessage;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class PutPutReplyMessageJUnitTest extends OldValueImporterTestBase {
+
+  @Override
+  protected OldValueImporter createImporter() {
+    return new PutReplyMessage(1, true, Operation.PUT_IF_ABSENT, null, null, null);
+  }
+
+  @Override
+  protected Object getOldValueFromImporter(OldValueImporter ovi) {
+    return ((PutReplyMessage)ovi).getOldValue();
+  }
+
+  @Override
+  protected void toData(OldValueImporter ovi, HeapDataOutputStream hdos) throws IOException {
+    ((PutReplyMessage)ovi).toData(hdos);
+  }
+
+  @Override
+  protected void fromData(OldValueImporter ovi, byte[] bytes) throws IOException, ClassNotFoundException {
+    ((PutReplyMessage)ovi).fromData(new DataInputStream(new ByteArrayInputStream(bytes)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
new file mode 100644
index 0000000..2b5f5bf
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/OffHeapWriteObjectAsByteArrayJUnitTest.java
@@ -0,0 +1,101 @@
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.internal.offheap.NullOffHeapMemoryStats;
+import com.gemstone.gemfire.internal.offheap.NullOutOfOffHeapMemoryListener;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
+import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.DataAsAddress;
+import com.gemstone.gemfire.internal.offheap.StoredObject;
+import com.gemstone.gemfire.internal.offheap.UnsafeMemoryChunk;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class OffHeapWriteObjectAsByteArrayJUnitTest {
+
+  @Before
+  public void setUp() throws Exception {
+    SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{new UnsafeMemoryChunk(1024*1024)});
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+  }
+  
+  private StoredObject createStoredObject(byte[] bytes, boolean isSerialized, boolean isCompressed) {
+    return SimpleMemoryAllocatorImpl.getAllocator().allocateAndInitialize(bytes, isSerialized, isCompressed, null);
+  }
+  
+  private DataInputStream createInput(HeapDataOutputStream hdos) {
+    ByteArrayInputStream bais = new ByteArrayInputStream(hdos.toByteArray());
+    return new DataInputStream(bais);
+  }
+  
+  @Test
+  public void testByteArrayChunk() throws IOException, ClassNotFoundException {
+    byte[] expected = new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16};
+    StoredObject so = createStoredObject(expected, false, false);
+    assertTrue(so instanceof Chunk);
+    HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
+    DataSerializer.writeObjectAsByteArray(so, hdos);
+    DataInputStream in = createInput(hdos);
+    byte[] actual = DataSerializer.readByteArray(in);
+    assertArrayEquals(expected, actual);
+  }
+  
+  @Test
+  public void testByteArrayDataAsAddress() throws IOException, ClassNotFoundException {
+    byte[] expected = new byte[] {1, 2, 3};
+    StoredObject so = createStoredObject(expected, false, false);
+    assertTrue(so instanceof DataAsAddress);
+    HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
+    DataSerializer.writeObjectAsByteArray(so, hdos);
+    DataInputStream in = createInput(hdos);
+    byte[] actual = DataSerializer.readByteArray(in);
+    assertArrayEquals(expected, actual);
+  }
+  
+  @Test
+  public void testStringChunk() throws IOException, ClassNotFoundException {
+    byte[] expected = EntryEventImpl.serialize("1234567890");
+    StoredObject so = createStoredObject(expected, true, false);
+    assertTrue(so instanceof Chunk);
+    HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
+    DataSerializer.writeObjectAsByteArray(so, hdos);
+    DataInputStream in = createInput(hdos);
+    byte[] actual = DataSerializer.readByteArray(in);
+    assertArrayEquals(expected, actual);
+    assertNoMoreInput(in);
+  }
+  
+  @Test
+  public void testStringDataAsAddress() throws IOException, ClassNotFoundException {
+    byte[] expected = EntryEventImpl.serialize("1234");
+    StoredObject so = createStoredObject(expected, true, false);
+    assertTrue(so instanceof DataAsAddress);
+    HeapDataOutputStream hdos = new HeapDataOutputStream(new byte[1024]);
+    DataSerializer.writeObjectAsByteArray(so, hdos);
+    DataInputStream in = createInput(hdos);
+    byte[] actual = DataSerializer.readByteArray(in);
+    assertArrayEquals(expected, actual);
+  }
+  
+  private void assertNoMoreInput(DataInputStream in) throws IOException {
+    assertEquals(0, in.available());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java
index 0ef1bed..fc05868 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/ByteSourceJUnitTest.java
@@ -19,9 +19,9 @@ import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSource;
 import com.gemstone.gemfire.internal.tcp.ByteBufferInputStream.ByteSourceFactory;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
-@Category(IntegrationTest.class)
+@Category(UnitTest.class)
 public class ByteSourceJUnitTest {
   
   protected ByteSource createByteSource(byte[] bytes) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d8570c98/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index 4dd927d..79f5e22 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -1244,7 +1244,7 @@ toData,252,2a03b500082a2bb700852ab600862bb800872ab400043d2ab40024c600091cb200758
 
 com/gemstone/gemfire/internal/cache/RemotePutMessage$PutReplyMessage,2
 fromData,81,2a2bb700272bb9002801001100ff7e913d2a1c047e99000704a7000403b500032a2bb900280100b80029b500022a2bb8002ab500061c057e9900181c077e99000704a70004033e2a1d2bb8002bb50007b1
-toData,118,2a2bb7002c033d2ab400039900081c0480913d2ab40007c600081c0580913d2ab40007c1002d9900081c0780913d2b1cb9002e02002b2ab40002b4002fb9002e02002ab400309900182ab40006c00031c000314e2b2db900320200a7000b2ab400062bb800332ab40007c6000b2ab400072bb80034b1
+toData,94,2a2bb70032033d2ab400039900081c0480913d2ab40007c600081c0580913d2ab40007c100339900081c0780913d2b1cb9003402002b2ab40002b40035b9003402002b2ab600362ab40037b800382ab40007c6000b2ab400072bb80039b1
 
 com/gemstone/gemfire/internal/cache/RemoteRegionOperation,2
 fromData,20,2a2bb700292ab8002a2bb9002b010032b5000cb1
@@ -1770,7 +1770,7 @@ toData,374,014d2ab4001fb60072b9007301003e2ab40009b60074c600161d9900122ab40008990
 
 com/gemstone/gemfire/internal/cache/partitioned/PutMessage$PutReplyMessage,2
 fromData,48,2a2bb700242a2bb900250100b500032a2bb900260100b80027b500022a2bb80028b500062a2bb80028c00029b50007b1
-toData,45,2a2bb7002a2b2ab40003b9002b02002b2ab40002b4002cb9002d02002ab400062bb8002e2ab400072bb8002eb1
+toData,54,2a2bb7002a2b2ab40003b9002b02002b2ab40002b4002cb9002d02002ab6002e4d2b2ab6002e2ab4002fb800302ab400072bb80031b1
 
 com/gemstone/gemfire/internal/cache/partitioned/QueryMessage,2
 fromData,63,2a2bb7007b2a2bb8007cb5000a2a2bb8007db5000b2a2bb8007eb5000c2a2bb8007fb60080b5000e2a2bb8007fb60080b500792a2bb8007fb60080b50011b1



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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeWriterDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeWriterDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeWriterDUnitTest.java
deleted file mode 100644
index bb5850d..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeWriterDUnitTest.java
+++ /dev/null
@@ -1,418 +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.cache30;
-
-import java.io.IOException;
-import java.util.Properties;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.cache.client.SubscriptionNotEnabledException;
-
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.VM;
-
-/**
- * Tests the BridgeWriter API, including interaction with Region.
- *
- * @author Kirk Lund
- * @since 4.2.3
- */
-public class BridgeWriterDUnitTest extends BridgeTestCase {
-
-  public BridgeWriterDUnitTest(String name) {
-    super(name);
-  }
-  
-  public void tearDown2() throws Exception {
-    super.tearDown2();
-    disconnectAllFromDS(); // cleans up bridge server and client and lonerDS
-  }
-  
-  /**
-   * Tests for Bug 35381 Calling register interest if 
-   * establishCallbackConnection is not set causes bridge server NPE.
-   */
-  public void testBug35381() throws Exception {
-    final Host host = Host.getHost(0);
-    final String name = this.getUniqueName();
-    final int[] ports = new int[1]; // 1 server in this test
-    
-    final int whichVM = 0;
-    final VM vm = Host.getHost(0).getVM(whichVM);
-    vm.invoke(new CacheSerializableRunnable("Create bridge server") {
-      public void run2() throws CacheException {
-        getLogWriter().info("[testBug35381] Create BridgeServer");
-        getSystem();
-        AttributesFactory factory = new AttributesFactory();
-        factory.setScope(Scope.LOCAL);
-        Region region = createRegion(name, factory.create());
-        assertNotNull(region);
-        assertNotNull(getRootRegion().getSubregion(name));
-        region.put("KEY-1", "VAL-1");
-        
-        try {
-          bridgeServerPort = startBridgeServer(0);
-        }
-        catch (IOException e) {
-          getLogWriter().error("startBridgeServer threw IOException", e);
-          fail("startBridgeServer threw IOException " + e.getMessage());
-        }
-        
-        assertTrue(bridgeServerPort != 0);
-    
-        getLogWriter().info("[testBug35381] port=" + bridgeServerPort);
-        getLogWriter().info("[testBug35381] serverMemberId=" + getMemberId());
-      }
-    });
-    ports[whichVM] = vm.invokeInt(BridgeWriterDUnitTest.class, 
-                                  "getBridgeServerPort");
-    assertTrue(ports[whichVM] != 0);
-    
-    getLogWriter().info("[testBug35381] create bridge client");
-    Properties config = new Properties();
-    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    getSystem(config);
-    getCache();
-    
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-
-    getLogWriter().info("[testBug35381] creating connection pool");
-    boolean establishCallbackConnection = false; // SOURCE OF BUG 35381
-    BridgeTestCase.configureConnectionPool(factory, getServerHostName(host), ports, establishCallbackConnection, -1, -1, null);
-    Region region = createRegion(name, factory.create());
-    assertNotNull(getRootRegion().getSubregion(name));
-    try {
-      region.registerInterest("KEY-1");
-      fail("registerInterest failed to throw BridgeWriterException with establishCallbackConnection set to false"); 
-    }
-    catch (SubscriptionNotEnabledException expected) {
-    }
-  }
-  protected static int bridgeServerPort;
-  private static int getBridgeServerPort() {
-    return bridgeServerPort;
-  }
-  
-  /**
-   * Tests failover of register interest from client point of view. Related
-   * bugs include:
-   *
-   * <p>Bug 35654 "failed re-registration may never be detected and thus
-   * may never re-re-register"
-   *
-   * <p>Bug 35639 "registerInterest re-registration happens everytime a healthy
-   * server is detected"
-   *
-   * <p>Bug 35655 "a single failed re-registration causes all other pending
-   * re-registrations to be cancelled"
-   */
-  public void _testRegisterInterestFailover() throws Exception {
-    // controller is bridge client
-    
-    final Host host = Host.getHost(0);
-    final String name = this.getUniqueName();
-    final String regionName1 = name+"-1";
-    final String regionName2 = name+"-2";
-    final String regionName3 = name+"-3";
-    final String key1 = "KEY-"+regionName1+"-1";
-    final String key2 = "KEY-"+regionName1+"-2";
-    final String key3 = "KEY-"+regionName1+"-3";
-    final int[] ports = new int[3]; // 3 servers in this test
-    
-    // create first bridge server with region for client...
-    final int firstServerIdx = 0;
-    final VM firstServerVM = Host.getHost(0).getVM(firstServerIdx);
-    firstServerVM.invoke(new CacheSerializableRunnable("Create first bridge server") {
-      public void run2() throws CacheException {
-        getLogWriter().info("[testRegisterInterestFailover] Create first bridge server");
-        getSystem();
-        AttributesFactory factory = new AttributesFactory();
-        factory.setScope(Scope.LOCAL);
-        Region region1 = createRootRegion(regionName1, factory.create());
-        Region region2 = createRootRegion(regionName2, factory.create());
-        Region region3 = createRootRegion(regionName3, factory.create());
-        region1.put(key1, "VAL-1");
-        region2.put(key2, "VAL-1");
-        region3.put(key3, "VAL-1");
-        
-        try {
-          bridgeServerPort = startBridgeServer(0);
-        }
-        catch (IOException e) {
-          getLogWriter().error("startBridgeServer threw IOException", e);
-          fail("startBridgeServer threw IOException " + e.getMessage());
-        }
-        
-        assertTrue(bridgeServerPort != 0);
-    
-        getLogWriter().info("[testRegisterInterestFailover] " +
-          "firstServer port=" + bridgeServerPort);
-        getLogWriter().info("[testRegisterInterestFailover] " +
-          "firstServer memberId=" + getMemberId());
-      }
-    });
-
-    // create second bridge server missing region for client...
-    final int secondServerIdx = 1;
-    final VM secondServerVM = Host.getHost(0).getVM(secondServerIdx);
-    secondServerVM.invoke(new CacheSerializableRunnable("Create second bridge server") {
-      public void run2() throws CacheException {
-        getLogWriter().info("[testRegisterInterestFailover] Create second bridge server");
-        getSystem();
-        AttributesFactory factory = new AttributesFactory();
-        factory.setScope(Scope.LOCAL);
-        Region region1 = createRootRegion(regionName1, factory.create());
-        Region region3 = createRootRegion(regionName3, factory.create());
-        region1.put(key1, "VAL-2");
-        region3.put(key3, "VAL-2");
-        
-        try {
-          bridgeServerPort = startBridgeServer(0);
-        }
-        catch (IOException e) {
-          getLogWriter().error("startBridgeServer threw IOException", e);
-          fail("startBridgeServer threw IOException " + e.getMessage());
-        }
-        
-        assertTrue(bridgeServerPort != 0);
-    
-        getLogWriter().info("[testRegisterInterestFailover] " +
-          "secondServer port=" + bridgeServerPort);
-        getLogWriter().info("[testRegisterInterestFailover] " +
-          "secondServer memberId=" + getMemberId());
-      }
-    });
-
-    // get the bridge server ports...
-    ports[firstServerIdx] = firstServerVM.invokeInt(
-      BridgeWriterDUnitTest.class, "getBridgeServerPort");
-    assertTrue(ports[firstServerIdx] != 0);
-    ports[secondServerIdx] = secondServerVM.invokeInt(
-      BridgeWriterDUnitTest.class, "getBridgeServerPort");
-    assertTrue(ports[secondServerIdx] != 0);
-    assertTrue(ports[firstServerIdx] != ports[secondServerIdx]);
-    
-    // stop second and third servers
-    secondServerVM.invoke(new CacheSerializableRunnable("Stop second bridge server") {
-      public void run2() throws CacheException {
-        stopBridgeServers(getCache());
-      }
-    });
-    
-    // create the bridge client
-    getLogWriter().info("[testBug35654] create bridge client");
-    Properties config = new Properties();
-    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    getSystem(config);
-    getCache();
-    
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-
-    getLogWriter().info("[testRegisterInterestFailover] creating connection pool");
-    boolean establishCallbackConnection = true;
-    final PoolImpl p = (PoolImpl)BridgeTestCase.configureConnectionPool(factory, getServerHostName(host), ports, establishCallbackConnection, -1, -1, null);
-
-    final Region region1 = createRootRegion(regionName1, factory.create());
-    final Region region2 = createRootRegion(regionName2, factory.create());
-    final Region region3 = createRootRegion(regionName3, factory.create());
-
-    assertTrue(region1.getInterestList().isEmpty());
-    assertTrue(region2.getInterestList().isEmpty());
-    assertTrue(region3.getInterestList().isEmpty());
-
-    region1.registerInterest(key1);
-    region2.registerInterest(key2);
-    region3.registerInterest(key3);
-
-    assertTrue(region1.getInterestList().contains(key1));
-    assertTrue(region2.getInterestList().contains(key2));
-    assertTrue(region3.getInterestList().contains(key3));
-    
-    assertTrue(region1.getInterestListRegex().isEmpty());
-    assertTrue(region2.getInterestListRegex().isEmpty());
-    assertTrue(region3.getInterestListRegex().isEmpty());
-    
-    // get ConnectionProxy and wait until connected to first server
-    WaitCriterion ev = new WaitCriterion() {
-      public boolean done() {
-        return p.getPrimaryPort() != -1;
-      }
-      public String description() {
-        return "primary port remained invalid";
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 10 * 1000, 200, true);
-    assertEquals(ports[firstServerIdx], p.getPrimaryPort()); 
-    
-    // assert intial values
-    assertEquals("VAL-1", region1.get(key1));
-    assertEquals("VAL-1", region2.get(key2));
-    assertEquals("VAL-1", region3.get(key3));
-    
-    // do puts on server1 and make sure values come thru for all 3 registrations
-    firstServerVM.invoke(new CacheSerializableRunnable("Puts from first bridge server") {
-      public void run2() throws CacheException {
-        Region region1 = getCache().getRegion(regionName1);
-        region1.put(key1, "VAL-1-1");
-        Region region2 = getCache().getRegion(regionName2);
-        region2.put(key2, "VAL-1-1");
-        Region region3 = getCache().getRegion(regionName3);
-        region3.put(key3, "VAL-1-1");
-      }
-    });
-
-    ev = new WaitCriterion() {
-      public boolean done() {
-        if (!"VAL-1-1".equals(region1.get(key1)) || 
-            !"VAL-1-1".equals(region2.get(key2)) ||
-            !"VAL-1-1".equals(region3.get(key3))
-            ) return  false;
-        return true;
-      }
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 10 * 1000, 200, true);
-    assertEquals("VAL-1-1", region1.get(key1));
-    assertEquals("VAL-1-1", region2.get(key2));
-    assertEquals("VAL-1-1", region3.get(key3));
-    
-    // force failover to server 2
-    secondServerVM.invoke(new CacheSerializableRunnable("Start second bridge server") {
-      public void run2() throws CacheException {
-        try {
-          startBridgeServer(ports[secondServerIdx]);
-        }
-        catch (IOException e) {
-          getLogWriter().error("startBridgeServer threw IOException", e);
-          fail("startBridgeServer threw IOException " + e.getMessage());
-        }
-      }
-    });
-   
-    firstServerVM.invoke(new CacheSerializableRunnable("Stop first bridge server") {
-      public void run2() throws CacheException {
-        stopBridgeServers(getCache());
-      }
-    });
-
-    // wait for failover to second server
-    ev = new WaitCriterion() {
-      public boolean done() {
-        return ports[secondServerIdx] == p.getPrimaryPort();
-      }
-      public String description() {
-        return "primary port never became " + ports[secondServerIdx];
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 100 * 1000, 200, true);
-    
-    try {
-      assertEquals(null, region2.get(key2));
-      fail("CacheLoaderException expected");
-    }
-    catch (com.gemstone.gemfire.cache.CacheLoaderException e) {
-    }
-  
-    // region2 registration should be gone now
-    // do puts on server2 and make sure values come thru for only 2 registrations
-    secondServerVM.invoke(new CacheSerializableRunnable("Puts from second bridge server") {
-      public void run2() throws CacheException {
-        AttributesFactory factory = new AttributesFactory();
-        factory.setScope(Scope.LOCAL);
-        createRootRegion(regionName2, factory.create());
-      }
-    });
-    
-    // assert that there is no actively registered interest on region2
-    assertTrue(region2.getInterestList().isEmpty());
-    assertTrue(region2.getInterestListRegex().isEmpty());
-
-    region2.put(key2, "VAL-0");
-    
-    secondServerVM.invoke(new CacheSerializableRunnable("Put from second bridge server") {
-      public void run2() throws CacheException {
-        Region region1 = getCache().getRegion(regionName1);
-        region1.put(key1, "VAL-2-2");
-        Region region2 = getCache().getRegion(regionName2);
-        region2.put(key2, "VAL-2-1");
-        Region region3 = getCache().getRegion(regionName3);
-        region3.put(key3, "VAL-2-2");
-      }
-    });
-    
-    // wait for updates to come thru
-    ev = new WaitCriterion() {
-      public boolean done() {
-        if (!"VAL-2-2".equals(region1.get(key1)) || 
-            !"VAL-2-2".equals(region3.get(key3)))
-          return false;
-        return true;
-      }
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 100 * 1000, 200, true);
-    assertEquals("VAL-2-2", region1.get(key1));
-    assertEquals("VAL-0",   region2.get(key2));
-    assertEquals("VAL-2-2", region3.get(key3));
-
-    // assert again that there is no actively registered interest on region2
-    assertTrue(region2.getInterestList().isEmpty());
-
-    // register interest again on region2 and make
-    region2.registerInterest(key2);
-    assertEquals("VAL-2-1", region2.get(key2));
-    
-    secondServerVM.invoke(new CacheSerializableRunnable("Put from second bridge server") {
-      public void run2() throws CacheException {
-        Region region1 = getCache().getRegion(regionName1);
-        region1.put(key1, "VAL-2-3");
-        Region region2 = getCache().getRegion(regionName2);
-        region2.put(key2, "VAL-2-2");
-        Region region3 = getCache().getRegion(regionName3);
-        region3.put(key3, "VAL-2-3");
-      }
-    });
-    
-    // wait for updates to come thru
-    ev = new WaitCriterion() {
-      public boolean done() {
-        if (!"VAL-2-3".equals(region1.get(key1)) || 
-            !"VAL-2-2".equals(region2.get(key2)) ||
-            !"VAL-2-3".equals(region3.get(key3)))
-          return false;
-        return true;
-      }
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 100 * 1000, 200, true);
-    assertEquals("VAL-2-3", region1.get(key1));
-    assertEquals("VAL-2-2", region2.get(key2));
-    assertEquals("VAL-2-3", region3.get(key3));
-
-    // assert public methods report actively registered interest on region2
-    assertTrue(region2.getInterestList().contains(key2));
-  }
-  
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeWriterSelectorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeWriterSelectorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeWriterSelectorDUnitTest.java
deleted file mode 100644
index 147af46..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeWriterSelectorDUnitTest.java
+++ /dev/null
@@ -1,16 +0,0 @@
-package com.gemstone.gemfire.cache30;
-
-/**
- * Same as BridgeWriterDUnitTest but uses selector in server
- *
- * @author darrel
- * @since 5.1
- */
-public class BridgeWriterSelectorDUnitTest extends BridgeWriterDUnitTest {
-  public BridgeWriterSelectorDUnitTest(String name) {
-    super(name);
-  }
-  protected int getMaxThreads() {
-    return 2;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java
index 487434b..cfeb0e1 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java
@@ -27,7 +27,7 @@ import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.NanoTimer;
-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.BucketRegion.RawValue;
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
@@ -51,7 +51,7 @@ import dunit.VM;
  * @author Mitch Thomas
  * @since bugfix5.7
  */
-public class Bug38741DUnitTest extends BridgeTestCase {
+public class Bug38741DUnitTest extends ClientServerTestCase {
   private static final long serialVersionUID = 1L;
 
   protected RegionAttributes getRegionAttributes() {
@@ -151,7 +151,7 @@ public class Bug38741DUnitTest extends BridgeTestCase {
         getCache();
         
         AttributesFactory factory = new AttributesFactory();
-        BridgeTestCase.configureConnectionPool(factory, serverHostName, ports, true,-1,1,null);
+        ClientServerTestCase.configureConnectionPool(factory, serverHostName, ports, true,-1,1,null);
         factory.setScope(Scope.LOCAL);
         Region r = createRootRegion(rName, factory.create());
         SerializationCountingKey ks1 = new SerializationCountingKey(k1);
@@ -177,8 +177,8 @@ public class Bug38741DUnitTest extends BridgeTestCase {
     server.invoke(new CacheSerializableRunnable("Assert copy behavior after client is setup") {
       public void run2() throws CacheException {
         Region r = getRootRegion(rName);
-        BridgeServerImpl bsi = (BridgeServerImpl)
-          getCache().getBridgeServers().iterator().next();
+        CacheServerImpl bsi = (CacheServerImpl)
+          getCache().getCacheServers().iterator().next();
         Collection cp = bsi.getAcceptor().getCacheClientNotifier().getClientProxies();
         // Should only be one because only one client is connected
         assertEquals(1, cp.size());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java
index 95cc02a..e06cd13 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml30DUnitTest.java
@@ -39,7 +39,6 @@ import com.gemstone.gemfire.cache.MirrorType;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.util.BridgeClient;
 import com.gemstone.gemfire.cache.util.ObjectSizer;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
@@ -223,132 +222,6 @@ public class CacheXml30DUnitTest extends CacheXmlTestCase {
     testXml(cache);
   }
 
-  public static class TestBridgeClient extends BridgeClient {
-    public boolean equals(Object obj)
-    {
-      return obj instanceof BridgeClient; // needed for sameAs comparison
-    }
-  }
-  /**
-   * Tests a bridge client and special setCacheWriter behavior on 
-   * region attributes
-   */
-  public void testBridgeClientAsLoader() throws CacheException {
-    getSystem();
-    CacheCreation cache = new CacheCreation();
-    RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
-    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(1);
-
-    BridgeClient bc = new TestBridgeClient();
-    Properties props = new Properties();
-    props.setProperty("endpoints", "server=" + 
-      DistributedTestCase.getIPLiteral() + ":" + ports[0]);
-    bc.init(props);
-    
-    attrs.setCacheLoader(bc);
-
-    cache.createRegion("root", attrs);
-
-    addExpectedException("Connection refused: connect");
-    testXml(cache);
-    final Region ro = cache.getRegion("root");
-    assertSame(ro.getAttributes().getCacheLoader(), ro.getAttributes().getCacheWriter());
-    assertTrue(ro.getAttributes().getCacheLoader() instanceof TestBridgeClient);
-    // now that a BridgeLoader creates its own pool make sure it exists
-    assertEquals("pools="+PoolManager.getAll(), 1, PoolManager.getAll().size());
-  }
-
-  /**
-   * Tests a bridge client and special setCacheWriter over-ride behavior on 
-   * region attributes
-   */
-  public void testBridgeClientWriterOverride() throws CacheException {
-    getSystem();
-    CacheCreation cache = new CacheCreation();
-    RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
-    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(1);
-
-    BridgeClient bc = new TestBridgeClient();
-    Properties props = new Properties();
-    props.setProperty("endpoints", "server="
-      + DistributedTestCase.getIPLiteral() + ":" + ports[0]);
-    bc.init(props);
-    
-    attrs.setCacheLoader(bc);
-    attrs.setCacheWriter(new MyTestCacheWriter());
-
-    cache.createRegion("root", attrs);
-
-    addExpectedException("Connection refused: connect");
-    testXml(cache);
-    
-    final Region ro = cache.getRegion("root");
-    assertTrue(ro.getAttributes().getCacheLoader() instanceof BridgeClient);
-    assertTrue(ro.getAttributes().getCacheWriter() instanceof MyTestCacheWriter);
-    // now that a BridgeLoader creates its own pool make sure it exists
-    assertEquals("pools="+PoolManager.getAll(), 1, PoolManager.getAll().size());
-  }
-
-  
-  /**
-   * Tests a bridge client and special setCacheLoader over-ride behavior on 
-   * region attributes
-   */
-  public void testBridgeClientLoaderOverride() throws CacheException {
-    getSystem();
-    CacheCreation cache = new CacheCreation();
-    RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
-    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(1);
-
-    BridgeClient bc = new TestBridgeClient();
-    Properties props = new Properties();
-    props.setProperty("endpoints", "server=" 
-      + DistributedTestCase.getIPLiteral() + ":" + ports[0]);
-    bc.init(props);
-    
-    attrs.setCacheWriter(bc);
-    attrs.setCacheLoader(new CacheLoaderWithDeclarables());
-
-    cache.createRegion("root", attrs);
-
-    addExpectedException("Connection refused: connect");
-    testXml(cache);
-    final Region ro = cache.getRegion("root");
-    assertTrue(ro.getAttributes().getCacheWriter() instanceof BridgeClient);
-    assertTrue(ro.getAttributes().getCacheLoader() instanceof CacheLoaderWithDeclarables);
-    // now that a BridgeLoader creates its own pool make sure it exists
-    assertEquals("pools="+PoolManager.getAll(), 1, PoolManager.getAll().size());
-  }
-
-  /**
-   * Tests a bridge client and special setCacheWriter behavior on 
-   * region attributes
-   */
-  public void testBridgeClientAsWriter() throws CacheException {
-    getSystem();
-    CacheCreation cache = new CacheCreation();
-    RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
-    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(1);
-
-    BridgeClient bc = new TestBridgeClient();
-    Properties props = new Properties();
-    props.setProperty("endpoints", "server=" 
-      + DistributedTestCase.getIPLiteral() + ":" + ports[0]);
-    bc.init(props);
-    
-    attrs.setCacheWriter(bc);
-
-    cache.createRegion("root", attrs);
-
-    addExpectedException("Connection refused: connect");
-    testXml(cache);
-    final Region ro = cache.getRegion("root");
-    assertSame(ro.getAttributes().getCacheLoader(), ro.getAttributes().getCacheWriter());
-    assertTrue(ro.getAttributes().getCacheWriter() instanceof TestBridgeClient);
-    // now that a BridgeLoader creates its own pool make sure it exists
-    assertEquals("pools="+PoolManager.getAll(), 1, PoolManager.getAll().size());
-  }
-
   /**
    * Tests a cache writer with no parameters
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml40DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml40DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml40DUnitTest.java
index 686a0ab..acaf91a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml40DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml40DUnitTest.java
@@ -8,7 +8,7 @@
 package com.gemstone.gemfire.cache30;
 
 import com.gemstone.gemfire.cache.*;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.xmlcache.*;
 
@@ -56,9 +56,9 @@ public class CacheXml40DUnitTest extends CacheXml30DUnitTest {
   public void testBridgeServers() {
     CacheCreation cache = new CacheCreation();
 
-    BridgeServer bridge1 = cache.addBridgeServer();
+    CacheServer bridge1 = cache.addCacheServer();
     setBridgeAttributes(bridge1);
-    BridgeServer bridge2 = cache.addBridgeServer();
+    CacheServer bridge2 = cache.addCacheServer();
     bridge2.setPort(AvailablePortHelper.getRandomAvailableTCPPort());
 
     testXml(cache);
@@ -68,7 +68,7 @@ public class CacheXml40DUnitTest extends CacheXml30DUnitTest {
    * Used by testBridgeServers to set version specific attributes
    * @param bridge1 the bridge server to set attributes upon
    */
-  public void setBridgeAttributes(BridgeServer bridge1)
+  public void setBridgeAttributes(CacheServer bridge1)
   {
     bridge1.setPort(AvailablePortHelper.getRandomAvailableTCPPort());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
index f7aabf4..cd16cac 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml41DUnitTest.java
@@ -8,8 +8,9 @@
 package com.gemstone.gemfire.cache30;
 
 import com.gemstone.gemfire.cache.*;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.internal.cache.xmlcache.*;
+
 import java.io.*;
 
 import org.xml.sax.SAXException;
@@ -40,7 +41,7 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
   // ////// Test methods
 
   
-  public void setBridgeAttributes(BridgeServer bridge1)
+  public void setBridgeAttributes(CacheServer bridge1)
   {
     super.setBridgeAttributes(bridge1);
     bridge1.setMaximumTimeBetweenPings(12345);
@@ -548,7 +549,6 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
     assertEquals(true, DynamicRegionFactory.get().getConfig().getPersistBackup());
     assertEquals(true, DynamicRegionFactory.get().isOpen());
     assertEquals(null, DynamicRegionFactory.get().getConfig().getDiskDir());
-    assertEquals(null, DynamicRegionFactory.get().getConfig().getBridgeWriter());
     Region dr = getCache().getRegion("__DynamicRegions");    
     if(dr != null) {
         dr.localDestroyRegion();      
@@ -557,7 +557,7 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
   }
   public void testDynamicRegionFactoryNonDefault() throws CacheException {
     CacheCreation cache = new CacheCreation();
-    cache.setDynamicRegionFactoryConfig(new DynamicRegionFactory.Config((File)null, (com.gemstone.gemfire.cache.util.BridgeWriter)null, false, false));
+    cache.setDynamicRegionFactoryConfig(new DynamicRegionFactory.Config((File)null, null, false, false));
     RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
     cache.createRegion("root", attrs);
     // note that testXml can't check if they are same because enabling
@@ -567,7 +567,6 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
     assertEquals(false, DynamicRegionFactory.get().getConfig().getPersistBackup());
     assertEquals(true, DynamicRegionFactory.get().isOpen());
     assertEquals(null, DynamicRegionFactory.get().getConfig().getDiskDir());
-    assertEquals(null, DynamicRegionFactory.get().getConfig().getBridgeWriter());    
     Region dr = getCache().getRegion("__DynamicRegions");    
     if(dr != null) {
         dr.localDestroyRegion();      
@@ -582,7 +581,7 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
     CacheCreation cache = new CacheCreation();
     File f = new File("diskDir");
     f.mkdirs();
-    cache.setDynamicRegionFactoryConfig(new DynamicRegionFactory.Config(f, null));
+    cache.setDynamicRegionFactoryConfig(new DynamicRegionFactory.Config(f, null, true, true));
     RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
     cache.createRegion("root", attrs);
     // note that testXml can't check if they are same because enabling
@@ -590,28 +589,11 @@ public class CacheXml41DUnitTest extends CacheXml40DUnitTest
     testXml(cache, false);
     assertEquals(true, DynamicRegionFactory.get().isOpen());
     assertEquals(f.getAbsoluteFile(), DynamicRegionFactory.get().getConfig().getDiskDir());
-    assertEquals(null, DynamicRegionFactory.get().getConfig().getBridgeWriter());
     Region dr =getCache().getRegion("__DynamicRegions");    
     if(dr != null) {
         dr.localDestroyRegion();      
     }
   }
-  /**
-   * disabled test because it can only be done from a bridge client cache.
-   */
-  public void _testDynamicRegionFactoryCacheWriter() throws CacheException {
-    CacheCreation cache = new CacheCreation();
-    CacheWriter writer = new MyTestCacheWriter();
-    cache.setDynamicRegionFactoryConfig(new DynamicRegionFactory.Config(null, (com.gemstone.gemfire.cache.util.BridgeWriter)writer));
-    RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
-    cache.createRegion("root", attrs);
-    // note that testXml can't check if they are same because enabling
-    // dynamic regions causes a meta region to be produced.
-    testXml(cache, false);
-    assertEquals(true, DynamicRegionFactory.get().isOpen());
-    assertEquals(null, DynamicRegionFactory.get().getConfig().getDiskDir());
-    assertEquals(writer, DynamicRegionFactory.get().getConfig().getBridgeWriter());
-  }
 
   /**
    * Remove this override when bug #52052 is fixed.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml45DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml45DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml45DUnitTest.java
index bc41df8..9891e59 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml45DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml45DUnitTest.java
@@ -9,7 +9,7 @@ package com.gemstone.gemfire.cache30;
 
 import com.company.app.DBLoader;
 import com.gemstone.gemfire.cache.*;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.cache.DistributedRegion;
 import com.gemstone.gemfire.internal.cache.xmlcache.*;
@@ -46,7 +46,7 @@ public class CacheXml45DUnitTest extends CacheXml41DUnitTest {
   
 
   
-  public void setBridgeAttributes(BridgeServer bridge1)
+  public void setBridgeAttributes(CacheServer bridge1)
   {
     super.setBridgeAttributes(bridge1);
     bridge1.setMaxConnections(100);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml51DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml51DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml51DUnitTest.java
index 779b44d..59e024d 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml51DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml51DUnitTest.java
@@ -8,12 +8,12 @@
 package com.gemstone.gemfire.cache30;
 
 import com.gemstone.gemfire.cache.*;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.DiskWriteAttributesImpl;
 import com.gemstone.gemfire.internal.cache.xmlcache.*;
-import com.gemstone.gemfire.cache.util.BridgeServer;
 
 import dunit.Host;
 import dunit.VM;
@@ -148,7 +148,7 @@ public void testMessageSyncInterval() throws CacheException {
 public void testBridgeAttributesRelatedToClientQueuesHA() throws CacheException {
   CacheCreation cache = new CacheCreation();
   cache.setMessageSyncInterval(3445);
-  BridgeServer bs = cache.addBridgeServer();
+  CacheServer bs = cache.addCacheServer();
   bs.setMaximumMessageCount(12345);
   bs.setMessageTimeToLive(56789);
   bs.setPort(AvailablePortHelper.getRandomAvailableTCPPort());
@@ -158,7 +158,7 @@ public void testBridgeAttributesRelatedToClientQueuesHA() throws CacheException
   testXml(cache);
   Cache c = getCache();
   assertNotNull(c);
-  BridgeServer server = (BridgeServer)cache.getBridgeServers().iterator().next();
+  CacheServer server = (CacheServer)cache.getCacheServers().iterator().next();
   assertNotNull(server);
   assertEquals(12345,server.getMaximumMessageCount());
   assertEquals(56789,server.getMessageTimeToLive());     
@@ -199,7 +199,7 @@ public void testBridgeAttributesRelatedToClientQueuesHA() throws CacheException
   {
     CacheCreation cache = new CacheCreation();
 
-    BridgeServer bs = cache.addBridgeServer();
+    CacheServer bs = cache.addCacheServer();
     bs.setMaxThreads(37);
     bs.setMaxConnections(999);
     bs.setPort(AvailablePortHelper.getRandomAvailableTCPPort());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java
index 37cf277..85fcf2f 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/CacheXml57DUnitTest.java
@@ -376,7 +376,6 @@ public class CacheXml57DUnitTest extends CacheXml55DUnitTest
     assertEquals(false, DynamicRegionFactory.get().getConfig().getPersistBackup());
     assertEquals(true, DynamicRegionFactory.get().isOpen());
     assertEquals(null, DynamicRegionFactory.get().getConfig().getDiskDir());
-    assertEquals(null, DynamicRegionFactory.get().getConfig().getBridgeWriter());
     assertEquals("connectionPool", DynamicRegionFactory.get().getConfig().getPoolName());
     Region dr = getCache().getRegion("__DynamicRegions");    
     if(dr != null) {



[08/38] incubator-geode git commit: GEODE-244: Fixing race in testRecoverRedundancyWithOfflinePersistenceSimulation

Posted by bs...@apache.org.
GEODE-244: Fixing race in testRecoverRedundancyWithOfflinePersistenceSimulation

The test recreates the persistent partitioned region, and then asserts
that all buckets are recovered. But the buckets may be recovered
asynchronously. Now the test will wait for the buckets to show up.


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

Branch: refs/heads/feature/GEODE-77
Commit: 55cd9246fdcd6b79800caa77651871a127143fb3
Parents: 2810172
Author: Dan Smith <up...@apache.org>
Authored: Mon Oct 5 14:38:04 2015 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Mon Oct 5 14:38:04 2015 -0700

----------------------------------------------------------------------
 .../internal/cache/control/RebalanceOperationDUnitTest.java     | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/55cd9246/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
index 1b04ecb..31d889c 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
@@ -2783,8 +2783,11 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
       /*
        * No rebalancing above because the simulation flag is on.
        * Therefore, vm1 will have recovered its buckets.
+       * We need to wait for the buckets because they
+       * might still be in the middle of creation in the
+       * background
        */
-      assertEquals(vm1Buckets,getBucketList("region1",vm1));      
+      waitForBucketList("region1", vm1, vm1Buckets);      
     }
     
     // look at vm2 buckets


[28/38] incubator-geode git commit: [GEODE-384] Fix for failing tests: PRColocatedEquiJoinDUnitTest.testPRLocalQuerying and QueryREUpdateInProgressJUnitTest Refactor of GEODE-280 missed on comparing struct types

Posted by bs...@apache.org.
[GEODE-384] Fix for failing tests:
PRColocatedEquiJoinDUnitTest.testPRLocalQuerying and QueryREUpdateInProgressJUnitTest
Refactor of GEODE-280 missed on comparing struct types


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

Branch: refs/heads/feature/GEODE-77
Commit: 8087818d56b03784652898e4d6254516e660c2c6
Parents: 2eb4e17
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue Oct 6 10:50:24 2015 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Oct 6 10:53:17 2015 -0700

----------------------------------------------------------------------
 .../query/functional/StructSetOrResultsSet.java | 41 +++++++++++++++++++-
 1 file changed, 39 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8087818d/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java
index d53c28b..7f3e14d 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java
@@ -382,7 +382,7 @@ public class StructSetOrResultsSet extends TestCase {
       Object p1 = itert1.next();
       Object p2 = null;
       if (!checkOrder) {
-        if (!result2.contains(p1)) {
+        if (!collectionContains(result2, p1)) {
           fail("Atleast one element in the pair of SelectResults "
               + "supposedly identical, is not equal " + "Match not found for :"
               + p1 + " compared with:" + p2 + "; failed query=" + query + "; element unmatched ="
@@ -394,7 +394,7 @@ public class StructSetOrResultsSet extends TestCase {
         boolean matched = false;
         if (itert2.hasNext()) {
           p2 = itert2.next();
-          matched = (p2 == p1) || p2.equals(p1);
+          matched = objectsEqual(p1, p2);
           if (!matched) {
             fail("Order of results was not the same, match not found for :"
                 + p1 + " compared with:" + p2 + "; failed query=" + query + "; element unmatched ="
@@ -407,4 +407,41 @@ public class StructSetOrResultsSet extends TestCase {
       currentIndex ++;
     }
   }
+  
+  private boolean collectionContains(Collection collection, Object object) {
+    Iterator iterator = collection.iterator();
+    while (iterator.hasNext()) {
+      Object o = iterator.next();
+      if (objectsEqual(object, o)) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  private boolean objectsEqual(Object o1, Object o2) {
+    //Assumed that o1 and o2 are the same object type as both are from collections created by executing the same query
+    if (o1 instanceof Struct) {
+      //if o2 is null, an NPE will be thrown.
+      Object[] values1 = ((Struct) o1).getFieldValues();
+      Object[] values2 = ((Struct) o2).getFieldValues();
+      assertEquals(values1.length, values2.length);
+      boolean elementEqual = true;
+      for (int i = 0; i < values1.length; ++i) {
+        elementEqual = elementEqual
+            && ((values1[i] == values2[i]) || values1[i]
+                .equals(values2[i]));
+      }
+      if (elementEqual) {
+        return true;
+      }
+    } 
+    else {
+      //if o1 is null and o2 is not, an NPE will be thrown
+      if (o1 == o2 || o1.equals(o2)) {
+        return true;
+      }
+    }
+    return false;
+  }
 }


[09/38] incubator-geode git commit: GEODE-362: fix intermittent failures in TXExpiryJUnitTest

Posted by bs...@apache.org.
GEODE-362: fix intermittent failures in TXExpiryJUnitTest

The TXExpiryJUnitTest now uses a test hook so it can be notified
when an ExpiryTask has expired. Note that when a tx is in progress
ExpiryTasks expire but do not perform their expiration action.
When the tx completes the expiration is rescheduled.
TxExpiryJUnitTest now also configures millisecond expiration units
so it can run faster. And the code has been cleaned up to remove
all eclipse warnings.


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

Branch: refs/heads/feature/GEODE-77
Commit: 1673e2461524f97b93b413d4d1b60486c92b5752
Parents: 55cd924
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Mon Oct 5 16:12:35 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Oct 6 09:50:53 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/ExpiryTask.java      |  20 +-
 .../gemfire/internal/cache/LocalRegion.java     |   7 +
 .../com/gemstone/gemfire/TXExpiryJUnitTest.java | 510 +++++++++----------
 3 files changed, 270 insertions(+), 267 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1673e246/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
index d5dc5ee..5c428b2 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
@@ -222,7 +222,6 @@ public abstract class ExpiryTask extends SystemTimer.SystemTimerTask {
   
   protected final boolean expire(boolean isPending) throws CacheException 
   {
-    waitOnExpirationSuspension();
     ExpirationAction action = getAction();
     if (action == null) return false;
     return expire(action, isPending);
@@ -351,6 +350,7 @@ public abstract class ExpiryTask extends SystemTimer.SystemTimerTask {
           getLocalRegion().isDestroyed()) {
         return;
       }
+      waitOnExpirationSuspension();
       if (logger.isTraceEnabled()) {
         logger.trace("{} is fired", this);
       }
@@ -396,6 +396,10 @@ public abstract class ExpiryTask extends SystemTimer.SystemTimerTask {
        // is still usable:
        SystemFailure.checkFailure();
        logger.fatal(LocalizedMessage.create(LocalizedStrings.ExpiryTask_EXCEPTION_IN_EXPIRATION_TASK), ex);
+    } finally {
+      if (expiryTaskListener != null) {
+        expiryTaskListener.afterExpire(this);
+      }
     }
   }
 
@@ -508,4 +512,18 @@ public abstract class ExpiryTask extends SystemTimer.SystemTimerTask {
     return result;
   }
 
+  // Should only be set by unit tests
+  public static ExpiryTaskListener expiryTaskListener;
+  
+  /**
+   * Used by tests to determine if events related
+   * to an ExpiryTask have happened.
+   */
+  public interface ExpiryTaskListener {
+    /**
+     * Called after the given expiry task has expired.
+     */
+    public void afterExpire(ExpiryTask et);
+    
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1673e246/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index 617a7ec..7c1ec89 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -8822,6 +8822,13 @@ public class LocalRegion extends AbstractRegion
   public RegionIdleExpiryTask getRegionIdleExpiryTask() {
     return this.regionIdleExpiryTask;
   }
+  /**
+   * Used by unit tests to get access to the RegionTTLExpiryTask
+   * of this region. Returns null if no task exists.
+   */
+  public RegionTTLExpiryTask getRegionTTLExpiryTask() {
+    return this.regionTTLExpiryTask;
+  }
   
   private void addExpiryTask(RegionEntry re, boolean ifAbsent)
   {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1673e246/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
index da9623b..f8dffcc 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
@@ -7,18 +7,13 @@
  */
 package com.gemstone.gemfire;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
-import com.gemstone.gemfire.cache.*;
-import com.gemstone.gemfire.cache.util.*;
-import com.gemstone.gemfire.internal.OSProcess;
-import com.gemstone.gemfire.internal.cache.*;
-import com.gemstone.gemfire.internal.util.StopWatch;
-import com.gemstone.gemfire.distributed.*;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-import java.util.*;
+import java.util.Properties;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.junit.After;
@@ -26,6 +21,32 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.cache.AttributesMutator;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.CacheListener;
+import com.gemstone.gemfire.cache.CacheTransactionManager;
+import com.gemstone.gemfire.cache.CommitConflictException;
+import com.gemstone.gemfire.cache.EntryEvent;
+import com.gemstone.gemfire.cache.ExpirationAction;
+import com.gemstone.gemfire.cache.ExpirationAttributes;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionEvent;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
+import com.gemstone.gemfire.internal.cache.ExpiryTask;
+import com.gemstone.gemfire.internal.cache.ExpiryTask.ExpiryTaskListener;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.TXManagerImpl;
+import com.gemstone.gemfire.internal.cache.TXStateProxy;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import dunit.DistributedTestCase;
+import dunit.DistributedTestCase.WaitCriterion;
+
 /**
  * Tests transaction expiration functionality
  *
@@ -33,8 +54,7 @@ import org.junit.experimental.categories.Category;
  * @since 4.0
  *
  */
-@SuppressWarnings("deprecation")
-@Category(IntegrationTest.class)
+@Category(UnitTest.class)
 public class TXExpiryJUnitTest {
 
   protected GemFireCacheImpl cache;
@@ -43,9 +63,7 @@ public class TXExpiryJUnitTest {
   protected void createCache() throws CacheException {
     Properties p = new Properties();
     p.setProperty("mcast-port", "0"); // loner
-    this.cache = (GemFireCacheImpl)CacheFactory.create(DistributedSystem.connect(p));
-    AttributesFactory af = new AttributesFactory();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
+    this.cache = (GemFireCacheImpl) (new CacheFactory(p)).create();
     this.txMgr = this.cache.getCacheTransactionManager();
   }
   private void closeCache() {
@@ -56,7 +74,6 @@ public class TXExpiryJUnitTest {
         } catch (IllegalStateException ignore) {
         }
       }
-      // this.region = null;
       this.txMgr = null;
       Cache c = this.cache;
       this.cache = null;
@@ -72,319 +89,280 @@ public class TXExpiryJUnitTest {
   @After
   public void tearDown() throws Exception {
     closeCache();
-    InternalDistributedSystem ids = InternalDistributedSystem.getAnyInstance();
-    if (ids != null) {
-      ids.disconnect();
-    }
   }
 
   @Test
   public void testEntryTTLExpiration() throws CacheException {
-    AttributesFactory af = new AttributesFactory();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
-    af.setStatisticsEnabled(true);
-    af.setEntryTimeToLive(new ExpirationAttributes(2, ExpirationAction.DESTROY));
-    Region exprReg = this.cache.createRegion("TXEntryTTL", af.create());
-    generalEntryExpirationTest(exprReg, 2);
-    AttributesMutator mutator = exprReg.getAttributesMutator();
-    mutator.setEntryTimeToLive(new ExpirationAttributes(1, ExpirationAction.DESTROY));
-    generalEntryExpirationTest(exprReg, 1);
+    generalEntryExpirationTest(createRegion("TXEntryTTL"), new ExpirationAttributes(1, ExpirationAction.DESTROY), true);
   } 
 
   @Test
   public void testEntryIdleExpiration() throws CacheException {
-    AttributesFactory af = new AttributesFactory();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
-    af.setStatisticsEnabled(true);
-    af.setEntryIdleTimeout(new ExpirationAttributes(2, ExpirationAction.DESTROY));
-    Region exprReg = this.cache.createRegion("TXEntryIdle", af.create());
-//    exprReg.getCache().getLogger().info("invoking expiration test with 2");
-    generalEntryExpirationTest(exprReg, 2);
-    AttributesMutator mutator = exprReg.getAttributesMutator();
-    mutator.setEntryIdleTimeout(new ExpirationAttributes(1, ExpirationAction.DESTROY));
-//    exprReg.getCache().getLogger().info("invoking expiration test with 1 after setting idle timeout of 1 second");
-    generalEntryExpirationTest(exprReg, 1);
+    generalEntryExpirationTest(createRegion("TXEntryIdle"), new ExpirationAttributes(1, ExpirationAction.DESTROY), false);
   } 
   
-  private void waitDance(boolean list[], int waitMs) {
-    synchronized(list) {
-      if (!list[0]) {
-        try {
-          list.wait(waitMs);
-        }
-        catch (InterruptedException e) {
-          fail("Interrupted");
-        }
-        if (list[0]) {
-          fail("Cache listener detected a destroy... bad!");
-        }
-      } else {
-        fail("Cache listener detected a destroy oh man that is bad!");
-      }
-    }  
+  private Region<String, String> createRegion(String name) {
+    RegionFactory<String, String> rf = this.cache.createRegionFactory();
+    rf.setScope(Scope.DISTRIBUTED_NO_ACK);
+    rf.setStatisticsEnabled(true);
+    System.setProperty(LocalRegion.EXPIRY_MS_PROPERTY, "true");
+    try {
+      return rf.create(name);
+    } 
+    finally {
+      System.getProperties().remove(LocalRegion.EXPIRY_MS_PROPERTY);
+    }
   }
   
-  @SuppressWarnings("deprecation")
-  public void generalEntryExpirationTest(final Region exprReg, 
-                                         final int exprTime) 
+  public void generalEntryExpirationTest(final Region<String, String> exprReg, 
+                                         ExpirationAttributes exprAtt,
+                                         boolean useTTL) 
     throws CacheException 
   {
-    final int waitMs = exprTime * 1500;
-    final int patientWaitMs = exprTime * 90000;
+    final LocalRegion lr = (LocalRegion) exprReg;
     final boolean wasDestroyed[] = {false};
-    AttributesMutator mutator = exprReg.getAttributesMutator();
+    AttributesMutator<String, String> mutator = exprReg.getAttributesMutator();
     final AtomicInteger ac = new AtomicInteger();
     final AtomicInteger au = new AtomicInteger();
     final AtomicInteger ai = new AtomicInteger();
     final AtomicInteger ad = new AtomicInteger();
     
-//    exprReg.getCache().getLogger().info("generalEntryExpirationTest invoked with exprTime " + exprTime);
-
-    mutator.setCacheListener(new CacheListenerAdapter() {
-        public void close() {}
-        public void afterCreate(EntryEvent e) {
-//          e.getRegion().getCache().getLogger().info("invoked afterCreate for " + e);
-          ac.incrementAndGet();
-        }
-        public void afterUpdate(EntryEvent e) {
-//          e.getRegion().getCache().getLogger().info("invoked afterUpdate for " + e);
-          au.incrementAndGet();
-        }
-        public void afterInvalidate(EntryEvent e) {
-//          e.getRegion().getCache().getLogger().info("invoked afterInvalidate for " + e);
-          ai.incrementAndGet();
-        }
-        public void afterDestroy(EntryEvent e) {
-//          e.getRegion().getCache().getLogger().info("invoked afterDestroy for " + e);
-          ad.incrementAndGet();
-          if (e.getKey().equals("key0")) {
-            synchronized(wasDestroyed) {
-              wasDestroyed[0] = true;
-              wasDestroyed.notifyAll();
-            }
-          }
-        }
-        public void afterRegionInvalidate(RegionEvent event) {
-          fail("Unexpected invokation of afterRegionInvalidate");
-        }
-        public void afterRegionDestroy(RegionEvent event) {
-          if (!event.getOperation().isClose()) {
-            fail("Unexpected invokation of afterRegionDestroy");
-          }
-        }
-      });
-    
-    // Test to ensure an expriation does not cause a conflict
-    for(int i=0; i<2; i++) {
-      exprReg.put("key" + i, "value" + i);
-    }
-    try {  Thread.sleep(500); } catch (InterruptedException ie) {fail("interrupted");}
-    this.txMgr.begin();
-//    exprReg.getCache().getLogger().info("transactional update of key0");
-    exprReg.put("key0", "value");
-//    exprReg.getCache().getLogger().info("waiting for " + waitMs);
-    waitDance(wasDestroyed, waitMs);
-    assertEquals("value", exprReg.getEntry("key0").getValue());
-//    exprReg.getCache().getLogger().info("committing transaction");
-    try {
-      this.txMgr.commit();
-    } catch (CommitConflictException error) {
-      fail("Expiration should not cause commit to fail");
+    if (useTTL) {
+      mutator.setEntryTimeToLive(exprAtt);
+    } else {
+      mutator.setEntryIdleTimeout(exprAtt);
     }
-    assertEquals("value", exprReg.getEntry("key0").getValue());
-    try {
-      synchronized(wasDestroyed) {
-        if (!wasDestroyed[0]) {
-//          exprReg.getCache().getLogger().info("waiting for wasDestroyed to be set by listener");
-          long start = System.currentTimeMillis();
-          wasDestroyed.wait(patientWaitMs);
-          long took = System.currentTimeMillis()-start;
-          if (!wasDestroyed[0]) {
-//            exprReg.getCache().getLogger().info("wasDestroyed was never set by the listener");
-            OSProcess.printStacks(0, false);
-            fail("Cache listener did not detect a destroy in " + patientWaitMs + " ms! actuallyWaited "+took+"ms ac="+ac.get()+" au:"+au.get()+" ai:"+ai.get()+" ad:"+ad.get());
+    final CacheListener<String, String> cl = new CacheListenerAdapter<String, String>() {
+      public void afterCreate(EntryEvent<String, String> e) {
+        ac.incrementAndGet();
+      }
+      public void afterUpdate(EntryEvent<String, String> e) {
+        au.incrementAndGet();
+      }
+      public void afterInvalidate(EntryEvent<String, String> e) {
+        ai.incrementAndGet();
+      }
+      public void afterDestroy(EntryEvent<String, String> e) {
+        ad.incrementAndGet();
+        if (e.getKey().equals("key0")) {
+          synchronized(wasDestroyed) {
+            wasDestroyed[0] = true;
+            wasDestroyed.notifyAll();
           }
         }
       }
-    } catch (InterruptedException ie) {
-      fail("Caught InterruptedException while waiting for eviction");
-    }
-    assertTrue(!exprReg.containsKey("key0"));
-    // key1 is the canary for the rest of the entries
-    assertTrue(!exprReg.containsKey("key1"));
-
-    // rollback and failed commit test, ensure expiration continues
-    for(int j=0; j<2; j++) {
-      synchronized(wasDestroyed) {
-        wasDestroyed[0] = false;
+      public void afterRegionInvalidate(RegionEvent<String, String> event) {
+        fail("Unexpected invocation of afterRegionInvalidate");
       }
+      public void afterRegionDestroy(RegionEvent<String, String> event) {
+        if (!event.getOperation().isClose()) {
+          fail("Unexpected invocation of afterRegionDestroy");
+        }
+      }
+    };
+    mutator.addCacheListener(cl);
+    try {
+
+      ExpiryTask.suspendExpiration();
+      // Test to ensure an expiration does not cause a conflict
       for(int i=0; i<2; i++) {
         exprReg.put("key" + i, "value" + i);
       }
-      try {  Thread.sleep(500); } catch (InterruptedException ie) {fail("interrupted");}
       this.txMgr.begin();
       exprReg.put("key0", "value");
-      waitDance(wasDestroyed, waitMs);
+      waitForEntryExpiration(lr, "key0");
       assertEquals("value", exprReg.getEntry("key0").getValue());
-      String checkVal;
-      if (j==0) {
-        checkVal = "value0";
-        this.txMgr.rollback();
-      } else {
-        checkVal = "conflictVal";
-        final TXManagerImpl txMgrImpl = (TXManagerImpl)this.txMgr;
-        TXStateProxy tx = txMgrImpl.internalSuspend();
-        exprReg.put("key0", checkVal);
-        txMgrImpl.resume(tx);
-        try {
-          this.txMgr.commit();
-          fail("Expected CommitConflictException!");
-        } catch (CommitConflictException expected) {}
-      }
-//      exprReg.getCache().getLogger().info("waiting for listener to be invoked.  iteration = " + j);
       try {
-        synchronized(wasDestroyed) {
-          if (!wasDestroyed[0]) {
-            Object value = exprReg.get("key0");
-            if (value == null) { // destroy in progress?
-              wasDestroyed.wait(waitMs);
-              assertTrue(wasDestroyed[0]);
-            }
-            else {
-              assertEquals(checkVal, value);
-              long start = System.currentTimeMillis();
-              wasDestroyed.wait(patientWaitMs);
-              long took = System.currentTimeMillis()-start;
-              if (!wasDestroyed[0]) {
-                Map m = new HashMap(exprReg);
-                fail("Cache listener did not detect a destroy in " + patientWaitMs + " ms! actuallyWaited:"+took+"ms ac="+ac.get()+" au:"+au.get()+" ai:"+ai.get()+" ad:"+ad.get()+" j="+j+" region="+m);
-              }
-            }
-          } 
-        }
-      } catch (InterruptedException ie) {
-        fail("Caught InterruptedException while waiting for expiration");
+        ExpiryTask.suspendExpiration();
+        this.txMgr.commit();
+      } catch (CommitConflictException error) {
+        fail("Expiration should not cause commit to fail");
+      }
+      assertEquals("value", exprReg.getEntry("key0").getValue());
+      waitForEntryExpiration(lr, "key0");
+      synchronized(wasDestroyed) {
+        assertEquals(true, wasDestroyed[0]);
       }
       assertTrue(!exprReg.containsKey("key0"));
       // key1 is the canary for the rest of the entries
       assertTrue(!exprReg.containsKey("key1"));
+
+      // rollback and failed commit test, ensure expiration continues
+      for(int j=0; j<2; j++) {
+        synchronized(wasDestroyed) {
+          wasDestroyed[0] = false;
+        }
+        ExpiryTask.suspendExpiration();
+        for(int i=0; i<2; i++) {
+          exprReg.put("key" + i, "value" + i);
+        }
+        this.txMgr.begin();
+        exprReg.put("key0", "value");
+        waitForEntryExpiration(lr, "key0");
+        assertEquals("value", exprReg.getEntry("key0").getValue());
+        String checkVal;
+        ExpiryTask.suspendExpiration();
+        if (j==0) {
+          checkVal = "value0";
+          this.txMgr.rollback();
+        } else {
+          checkVal = "conflictVal";
+          final TXManagerImpl txMgrImpl = (TXManagerImpl)this.txMgr;
+          TXStateProxy tx = txMgrImpl.internalSuspend();
+          exprReg.put("key0", checkVal);
+          txMgrImpl.resume(tx);
+          try {
+            this.txMgr.commit();
+            fail("Expected CommitConflictException!");
+          } catch (CommitConflictException expected) {}
+        }
+        waitForEntryExpiration(lr, "key0");
+        synchronized(wasDestroyed) {
+          assertEquals(true, wasDestroyed[0]);
+        }
+        assertTrue(!exprReg.containsKey("key0"));
+        // key1 is the canary for the rest of the entries
+        assertTrue(!exprReg.containsKey("key1"));
+      }
+    } finally {
+      mutator.removeCacheListener(cl);
+      ExpiryTask.permitExpiration();
+    }
+  }
+  
+  private void waitForEntryExpiration(LocalRegion lr, String key) {
+    ExpirationDetector detector = new ExpirationDetector(lr.getEntryExpiryTask(key));
+    ExpiryTask.expiryTaskListener = detector;
+    try {
+      ExpiryTask.permitExpiration();
+      DistributedTestCase.waitForCriterion(detector, 3000, 2, true);
+    } finally {
+      ExpiryTask.expiryTaskListener = null;
+    }
+  }
+  private void waitForRegionExpiration(LocalRegion lr, boolean ttl) {
+    ExpirationDetector detector = new ExpirationDetector(ttl ? lr.getRegionTTLExpiryTask() : lr.getRegionIdleExpiryTask());
+    ExpiryTask.expiryTaskListener = detector;
+    try {
+      ExpiryTask.permitExpiration();
+      DistributedTestCase.waitForCriterion(detector, 3000, 2, true);
+    } finally {
+      ExpiryTask.expiryTaskListener = null;
+    }
+  }
+
+
+  /**
+   * Used to detect that a particular ExpiryTask has expired.
+   */
+  public static class ExpirationDetector implements ExpiryTaskListener, WaitCriterion {
+    private volatile boolean expired = false;
+    private final ExpiryTask et;
+    public ExpirationDetector(ExpiryTask et) {
+      assertNotNull(et);
+      this.et = et;
+    }
+    @Override
+    public void afterExpire(ExpiryTask et) {
+      if (et == this.et) {
+        this.expired = true;
+      }
+    }
+    @Override
+    public boolean done() {
+      return this.expired;
+    }
+    @Override
+    public String description() {
+      return "the expiry task " + this.et + " did not expire";
     }
   }
 
   @Test
   public void testRegionIdleExpiration() throws CacheException {
-    AttributesFactory af = new AttributesFactory();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
-    af.setStatisticsEnabled(true);
-    af.setRegionIdleTimeout(new ExpirationAttributes(2, ExpirationAction.INVALIDATE));
-    Region exprReg = this.cache.createRegion("TXRegionIdle", af.create());
-    generalRegionExpirationTest(exprReg, 2, null, false);
-    generalRegionExpirationTest(exprReg, 1, new ExpirationAttributes(1, ExpirationAction.INVALIDATE), false);
-    generalRegionExpirationTest(exprReg, 2, new ExpirationAttributes(2, ExpirationAction.INVALIDATE), false);
-    generalRegionExpirationTest(exprReg, 1, new ExpirationAttributes(1, ExpirationAction.DESTROY), false);
+    Region<String, String> exprReg = createRegion("TXRegionIdle");
+    generalRegionExpirationTest(exprReg, new ExpirationAttributes(1, ExpirationAction.INVALIDATE), false);
+    generalRegionExpirationTest(exprReg, new ExpirationAttributes(1, ExpirationAction.DESTROY), false);
   } 
 
   @Test
   public void testRegionTTLExpiration() throws CacheException {
-    AttributesFactory af = new AttributesFactory();
-    af.setScope(Scope.DISTRIBUTED_NO_ACK);
-    af.setStatisticsEnabled(true);
-    af.setRegionTimeToLive(new ExpirationAttributes(1, ExpirationAction.INVALIDATE));
-    Region exprReg = this.cache.createRegion("TXRegionTTL", af.create());
-    generalRegionExpirationTest(exprReg, 1, null, true);
-    generalRegionExpirationTest(exprReg, 2, new ExpirationAttributes(2, ExpirationAction.INVALIDATE), true);
-    generalRegionExpirationTest(exprReg, 1, new ExpirationAttributes(1, ExpirationAction.INVALIDATE), true);
-    generalRegionExpirationTest(exprReg, 1, new ExpirationAttributes(1, ExpirationAction.DESTROY), true);
+    Region<String, String> exprReg = createRegion("TXRegionTTL");
+    generalRegionExpirationTest(exprReg, new ExpirationAttributes(1, ExpirationAction.INVALIDATE), true);
+    generalRegionExpirationTest(exprReg, new ExpirationAttributes(1, ExpirationAction.DESTROY), true);
   } 
   
-  private void generalRegionExpirationTest(final Region exprReg, 
-                                          final int exprTime, 
+  private void generalRegionExpirationTest(final Region<String, String> exprReg, 
                                           ExpirationAttributes exprAtt,
                                           boolean useTTL) 
     throws CacheException 
   {
-    final int waitMs = exprTime * 1500;
-    final int patientWaitMs = exprTime * 90000;
+    final LocalRegion lr = (LocalRegion) exprReg;
+    final ExpirationAction action = exprAtt.getAction();
     final boolean regionExpiry[] = {false};
-    AttributesMutator mutator = exprReg.getAttributesMutator();
-    mutator.setCacheListener(new CacheListenerAdapter() {
-        public void close() {}
-        public void afterCreate(EntryEvent e) {}
-        public void afterUpdate(EntryEvent e) {}
-        public void afterInvalidate(EntryEvent e) {}
-        public void afterDestroy(EntryEvent e) {}
-        public void afterRegionInvalidate(RegionEvent event) {
+    AttributesMutator<String, String> mutator = exprReg.getAttributesMutator();
+    final CacheListener<String, String> cl = new CacheListenerAdapter<String, String>() {
+      public void afterRegionInvalidate(RegionEvent<String, String> event) {
+        synchronized(regionExpiry) {
+          regionExpiry[0] = true;
+          regionExpiry.notifyAll();
+        }
+      }
+      public void afterRegionDestroy(RegionEvent<String, String> event) {
+        if (!event.getOperation().isClose()) {
           synchronized(regionExpiry) {
             regionExpiry[0] = true;
             regionExpiry.notifyAll();
           }
         }
-        public void afterRegionDestroy(RegionEvent event) {
-          if (!event.getOperation().isClose()) {
-            synchronized(regionExpiry) {
-              regionExpiry[0] = true;
-              regionExpiry.notifyAll();
-            }
-          }
-        }
-      });
-
-    // Create some keys and age them, I wish we could fake/force the age
-    // instead of having to actually wait
-    for(int i=0; i<2; i++) {
-      exprReg.put("key" + i, "value" + i);
-    }
-
-    ExpirationAction action;
-    if (exprAtt!=null) {
-      action = exprAtt.getAction();
-      if (useTTL) {
-        mutator.setRegionTimeToLive(exprAtt);
-      } else {
-        mutator.setRegionIdleTimeout(exprAtt);
       }
+    };
+    mutator.addCacheListener(cl);
+    if (useTTL) {
+      mutator.setRegionTimeToLive(exprAtt);
     } else {
-      if (useTTL) {
-        action = exprReg.getAttributes().getRegionTimeToLive().getAction();
-      } else {
-        action = exprReg.getAttributes().getRegionIdleTimeout().getAction();
-      }
+      mutator.setRegionIdleTimeout(exprAtt);
     }
 
-    // Potential race condition at this point if the Region operation
-    // is destroy i.e. we may not get to the transaction block
-    // before the destroy timer fires.
-
-    String regName = exprReg.getName();
-    // Test to ensure a region expriation does not cause a conflict    
-    this.txMgr.begin();
-    exprReg.put("key0", "value");
-    waitDance(regionExpiry, waitMs);
-    assertEquals("value", exprReg.getEntry("key0").getValue());
-    try {
-      this.txMgr.commit();
-    } catch (CommitConflictException error) {
-      fail("Expiration should not cause commit to fail");
-    }
     try {
+      ExpiryTask.suspendExpiration();
+
+      // Create some keys and age them, I wish we could fake/force the age
+      // instead of having to actually wait
+      for(int i=0; i<2; i++) {
+        exprReg.put("key" + i, "value" + i);
+      }
+
+      String regName = exprReg.getName();
+      // Test to ensure a region expiration does not cause a conflict    
+      this.txMgr.begin();
+      exprReg.put("key0", "value");
+      waitForRegionExpiration(lr, useTTL);
+      assertEquals("value", exprReg.getEntry("key0").getValue());
+      try {
+        ExpiryTask.suspendExpiration();
+        this.txMgr.commit();
+      } catch (CommitConflictException error) {
+        fail("Expiration should not cause commit to fail");
+      }
+      assertEquals("value", exprReg.getEntry("key0").getValue());
+      waitForRegionExpiration(lr, useTTL);
       synchronized(regionExpiry) {
-        if (!regionExpiry[0]) {
-          assertEquals("value", exprReg.getEntry("key0").getValue());
-          regionExpiry.wait(patientWaitMs);
-          if (!regionExpiry[0]) {
-            fail("Cache listener did not detect a region expiration in " + patientWaitMs + " ms!");
-          }
-        }
+        assertEquals(true, regionExpiry[0]);
       }
-    } catch (InterruptedException ie) {
-      fail("Caught InterruptedException while waiting for eviction");
-    }
-    if (action == ExpirationAction.DESTROY) {
-      assertNull("listener saw Region expiration, expected a destroy operation!", 
-                 this.cache.getRegion(regName));
-    } else {
-      assertTrue("listener saw Region expriation, expected invalidation", 
-                 !exprReg.containsValueForKey("key0"));
+      if (action == ExpirationAction.DESTROY) {
+        assertNull("listener saw Region expiration, expected a destroy operation!", 
+            this.cache.getRegion(regName));
+      } else {
+        assertTrue("listener saw Region expiration, expected invalidation", 
+            !exprReg.containsValueForKey("key0"));
+      }
+
+    } finally {
+      if (!exprReg.isDestroyed()) {
+        mutator.removeCacheListener(cl);
+      }
+      ExpiryTask.permitExpiration();
     }
 
     // @todo mitch test rollback and failed expiration


[38/38] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-77

Posted by bs...@apache.org.
Merge remote-tracking branch 'origin/develop' into feature/GEODE-77

Conflicts:
	gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java
	gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/jgroup/JGroupMembershipManager.java
	gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
	gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
	gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
	gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
	gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceWithUDPDUnitTest.java
	gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
	gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt


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

Branch: refs/heads/feature/GEODE-77
Commit: c4b14b7244fc021c08ed4a5753887c0a493f527f
Parents: 3e1a9f6 21743ec
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu Oct 15 09:05:33 2015 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Thu Oct 15 09:25:15 2015 -0700

----------------------------------------------------------------------
 README.md                                       |    2 +-
 build.gradle                                    |   11 +-
 gemfire-assembly/build.gradle                   |    9 +-
 gemfire-assembly/src/main/dist/bin/gfsh         |    2 +-
 gemfire-assembly/src/main/dist/bin/gfsh.bat     |    2 +-
 .../com/gemstone/gemfire/DataSerializer.java    |   11 +-
 .../gemfire/admin/GemFireMemberStatus.java      |   13 +-
 .../gemfire/admin/SystemMemberBridgeServer.java |    5 +-
 .../gemfire/admin/SystemMemberCache.java        |   27 -
 .../DistributedSystemHealthMonitor.java         |    2 +-
 .../admin/internal/SystemMemberCacheImpl.java   |   17 +-
 .../jmx/internal/SystemMemberCacheJmxImpl.java  |   13 -
 .../gemfire/cache/AttributesFactory.java        |   31 -
 .../java/com/gemstone/gemfire/cache/Cache.java  |   27 -
 .../gemfire/cache/DynamicRegionFactory.java     |  121 +-
 .../gemfire/cache/client/PoolFactory.java       |   10 +-
 .../cache/client/internal/AbstractOp.java       |    2 +-
 .../cache/client/internal/BridgePoolImpl.java   |  479 -----
 .../internal/BridgeServerLoadMessage.java       |   99 --
 .../client/internal/CacheServerLoadMessage.java |   99 ++
 .../cache/client/internal/DestroyOp.java        |    5 -
 .../client/internal/EndpointManagerImpl.java    |    8 +-
 .../internal/ExplicitConnectionSourceImpl.java  |   85 +-
 .../gemfire/cache/client/internal/GetOp.java    |    5 -
 .../gemfire/cache/client/internal/PoolImpl.java |   14 +-
 .../gemfire/cache/client/internal/PutOp.java    |    5 -
 .../cache/client/internal/QueueManagerImpl.java |   14 +-
 .../cache/client/internal/QueueStateImpl.java   |    6 +-
 .../internal/RegisterDataSerializersOp.java     |    8 +-
 .../internal/RegisterInstantiatorsOp.java       |    8 +-
 .../client/internal/ServerRegionProxy.java      |   44 +-
 .../internal/FunctionServiceManager.java        |    3 +-
 .../cache/query/internal/CompiledSelect.java    |   34 +-
 .../cache/query/internal/DefaultQuery.java      |    9 +-
 .../gemfire/cache/server/ServerLoadProbe.java   |   10 +-
 .../gemfire/cache/server/ServerMetrics.java     |   10 +-
 .../cache/server/internal/LoadMonitor.java      |   12 +-
 .../gemfire/cache/util/BridgeClient.java        |  156 --
 .../gemfire/cache/util/BridgeLoader.java        |  607 -------
 .../gemfire/cache/util/BridgeMembership.java    |   55 -
 .../cache/util/BridgeMembershipEvent.java       |   23 -
 .../cache/util/BridgeMembershipListener.java    |   41 -
 .../util/BridgeMembershipListenerAdapter.java   |   43 -
 .../gemfire/cache/util/BridgeServer.java        |  442 -----
 .../gemfire/cache/util/BridgeWriter.java        |  795 ---------
 .../cache/util/BridgeWriterException.java       |   48 -
 .../util/EndpointDoesNotExistException.java     |   34 -
 .../gemfire/cache/util/EndpointException.java   |   61 -
 .../cache/util/EndpointExistsException.java     |   28 -
 .../cache/util/EndpointInUseException.java      |   27 -
 .../util/IncompatibleVersionException.java      |   47 -
 .../util/ServerRefusedConnectionException.java  |   36 -
 .../UniversalMembershipListenerAdapter.java     |  352 ----
 .../cache/util/UnknownVersionException.java     |   39 -
 .../gemfire/cache/util/VersionException.java    |   45 -
 .../gemstone/gemfire/cache/util/package.html    |    3 -
 .../gemfire/distributed/ServerLauncher.java     |    4 +-
 .../FunctionExecutionPooledExecutor.java        |    7 +-
 .../internal/InternalDistributedSystem.java     |   12 +-
 .../internal/PooledExecutorWithDMStats.java     |    5 +-
 .../distributed/internal/ServerLocator.java     |   10 +-
 .../gemstone/gemfire/internal/DSFIDFactory.java |   12 +-
 .../internal/DataSerializableFixedID.java       |    6 +-
 .../internal/InternalDataSerializer.java        |   33 +-
 .../internal/admin/ClientStatsManager.java      |    7 -
 .../gemfire/internal/admin/GemFireVM.java       |    2 +-
 .../admin/remote/BridgeServerResponse.java      |   18 +-
 .../admin/remote/DurableClientInfoResponse.java |    4 +-
 .../admin/remote/RemoteBridgeServer.java        |   10 +-
 .../internal/admin/remote/RemoteCacheInfo.java  |    6 +-
 .../internal/admin/remote/RemoteGemFireVM.java  |    2 +-
 .../internal/cache/AbstractBridgeServer.java    |  425 -----
 .../internal/cache/AbstractCacheServer.java     |  398 +++++
 .../gemfire/internal/cache/AbstractRegion.java  |   52 -
 .../gemfire/internal/cache/BridgeObserver.java  |   89 -
 .../internal/cache/BridgeObserverAdapter.java   |  107 --
 .../internal/cache/BridgeObserverHolder.java    |   53 -
 .../internal/cache/BridgeRegionEventImpl.java   |  108 --
 .../internal/cache/BridgeServerAdvisor.java     |  165 --
 .../internal/cache/BridgeServerImpl.java        |  816 ---------
 .../gemfire/internal/cache/BucketAdvisor.java   |    2 +-
 .../gemfire/internal/cache/CacheConfig.java     |    8 +-
 .../internal/cache/CacheServerAdvisor.java      |  164 ++
 .../gemfire/internal/cache/CacheServerImpl.java |  812 +++++++++
 .../internal/cache/ClientRegionEventImpl.java   |  108 ++
 .../internal/cache/ClientServerObserver.java    |   90 +
 .../cache/ClientServerObserverAdapter.java      |  107 ++
 .../cache/ClientServerObserverHolder.java       |   53 +
 .../internal/cache/DestroyRegionOperation.java  |    6 +-
 .../cache/DistributedCacheOperation.java        |    8 +-
 .../cache/DistributedClearOperation.java        |    6 +-
 .../internal/cache/DistributedRegion.java       |    6 -
 .../gemfire/internal/cache/EntryExpiryTask.java |    3 +
 .../gemfire/internal/cache/ExpiryTask.java      |   60 +-
 .../cache/FindDurableQueueProcessor.java        |    4 +-
 .../internal/cache/GemFireCacheImpl.java        |   92 +-
 .../gemfire/internal/cache/GridAdvisor.java     |    6 +-
 .../gemfire/internal/cache/LocalRegion.java     |   48 +-
 .../internal/cache/PartitionedRegion.java       |    8 +-
 .../gemfire/internal/cache/PoolFactoryImpl.java |  171 +-
 .../gemfire/internal/cache/PoolManagerImpl.java |   19 -
 .../gemfire/internal/cache/RegionEventImpl.java |    2 +-
 .../internal/cache/RegionExpiryTask.java        |    3 +
 .../internal/cache/RemotePutMessage.java        |   56 +-
 .../internal/cache/doc-files/properties.html    |    4 +-
 .../cache/execute/InternalFunctionService.java  |    4 +-
 .../internal/cache/ha/HARegionQueue.java        |    6 +-
 .../PartitionedRegionObserverHolder.java        |    2 +-
 .../internal/cache/partitioned/PutMessage.java  |   21 +-
 .../internal/cache/tier/ConnectionProxy.java    |  160 +-
 .../cache/tier/InternalBridgeMembership.java    |  715 --------
 .../cache/tier/InternalClientMembership.java    |  617 +++++++
 .../cache/tier/sockets/AcceptorImpl.java        |   20 +-
 .../cache/tier/sockets/CacheClientNotifier.java |   16 +-
 .../cache/tier/sockets/CacheClientProxy.java    |    6 +-
 .../cache/tier/sockets/CacheClientUpdater.java  |   12 +-
 .../tier/sockets/ClientBlacklistProcessor.java  |    4 +-
 .../RemoveClientFromBlacklistMessage.java       |    4 +-
 .../cache/tier/sockets/ServerConnection.java    |    8 +-
 .../doc-files/communication-architecture.fig    |    4 +-
 .../cache/xmlcache/BridgeServerCreation.java    |  249 ---
 .../internal/cache/xmlcache/CacheCreation.java  |   47 +-
 .../cache/xmlcache/CacheServerCreation.java     |  238 +++
 .../cache/xmlcache/CacheXmlGenerator.java       |   41 +-
 .../internal/cache/xmlcache/CacheXmlParser.java |   18 +-
 .../cache/xmlcache/ClientCacheCreation.java     |    5 -
 .../xmlcache/RegionAttributesCreation.java      |    9 -
 .../gemfire/internal/i18n/LocalizedStrings.java |   32 +-
 .../internal/i18n/ParentLocalizedStrings.java   |   83 +-
 .../gemfire/internal/logging/LogService.java    |  128 +-
 .../internal/logging/log4j/Configurator.java    |    5 +
 .../offheap/OffHeapCachedDeserializable.java    |   20 +
 .../offheap/SimpleMemoryAllocatorImpl.java      |   36 +
 .../gemfire/internal/offheap/StoredObject.java  |   26 +-
 .../internal/beans/CacheServerBridge.java       |   18 +-
 .../beans/GatewayReceiverMBeanBridge.java       |    4 +-
 .../internal/beans/ManagementAdapter.java       |   42 +-
 .../management/internal/beans/ServerBridge.java |    6 +-
 .../cli/functions/ContunuousQueryFunction.java  |    4 +-
 .../functions/GetMemberInformationFunction.java |    4 +-
 .../internal/cli/i18n/CliStrings.java           |    2 +-
 .../management/membership/ClientMembership.java |    8 +-
 .../util/concurrent/SynchronousQueueNoSpin.java | 1144 ------------
 .../gemfire/cache/cache-9.0.xsd                 |   26 -
 .../internal/logging/log4j/log4j2-cli.xml       |   17 -
 .../internal/logging/log4j/log4j2-default.xml   |   22 -
 gemfire-core/src/main/resources/log4j2-cli.xml  |   17 +
 gemfire-core/src/main/resources/log4j2.xml      |   23 +
 .../batterytest/greplogs/ExpectedStrings.java   |    3 +-
 .../com/gemstone/gemfire/TXExpiryJUnitTest.java |  534 +++---
 .../cache/AttributesFactoryJUnitTest.java       |   18 -
 .../gemstone/gemfire/cache/ClientHelper.java    |    1 -
 .../cache/ConnectionPoolAndLoaderDUnitTest.java |    4 +-
 .../internal/AutoConnectionSourceDUnitTest.java |   23 +-
 .../AutoConnectionSourceWithUDPDUnitTest.java   |  140 ++
 .../internal/ConnectionPoolImplJUnitTest.java   |   11 +-
 .../internal/LocatorLoadBalancingDUnitTest.java |   10 +-
 .../cache/client/internal/LocatorTestBase.java  |   12 +-
 .../management/MemoryThresholdsDUnitTest.java   |    4 +-
 .../MemoryThresholdsOffHeapDUnitTest.java       |    4 +-
 .../gemfire/cache/query/QueryJUnitTest.java     |   91 +
 .../query/dunit/PdxStringQueryDUnitTest.java    |   16 +-
 .../query/dunit/QueryUsingPoolDUnitTest.java    |   12 +-
 .../cache/query/dunit/RemoteQueryDUnitTest.java |   90 +-
 ...esourceManagerWithQueryMonitorDUnitTest.java |    4 +-
 .../cache/query/functional/GroupByTestImpl.java |    2 +-
 .../query/functional/StructSetOrResultsSet.java |  260 ++-
 ...rrentIndexInitOnOverflowRegionDUnitTest.java |    4 +-
 .../cache30/BridgeMembershipDUnitTest.java      | 1644 ------------------
 .../BridgeMembershipSelectorDUnitTest.java      |   16 -
 .../gemfire/cache30/BridgeTestCase.java         |  376 ----
 .../gemfire/cache30/BridgeWriterDUnitTest.java  |  418 -----
 .../cache30/BridgeWriterSelectorDUnitTest.java  |   16 -
 .../gemfire/cache30/Bug38741DUnitTest.java      |   10 +-
 .../gemfire/cache30/CacheXml30DUnitTest.java    |  127 --
 .../gemfire/cache30/CacheXml40DUnitTest.java    |    8 +-
 .../gemfire/cache30/CacheXml41DUnitTest.java    |   28 +-
 .../gemfire/cache30/CacheXml45DUnitTest.java    |    4 +-
 .../gemfire/cache30/CacheXml51DUnitTest.java    |    8 +-
 .../gemfire/cache30/CacheXml57DUnitTest.java    |    1 -
 .../cache30/ClientMembershipDUnitTest.java      | 1644 ++++++++++++++++++
 .../ClientMembershipSelectorDUnitTest.java      |   16 +
 .../ClientRegisterInterestDUnitTest.java        |  418 +++++
 ...ClientRegisterInterestSelectorDUnitTest.java |   16 +
 .../gemfire/cache30/ClientServerTestCase.java   |  376 ++++
 .../AbstractServerLauncherJUnitTestCase.java    |    4 +-
 .../ServerLauncherLocalJUnitTest.java           |    4 +-
 .../ServerLauncherRemoteJUnitTest.java          |    8 +-
 .../internal/ProductUseLogDUnitTest.java        |    2 +-
 .../disttx/DistributedTransactionDUnitTest.java |    4 +-
 .../gemstone/gemfire/internal/FDDUnitTest.java  |   12 +-
 .../internal/cache/Bug39079DUnitTest.java       |    6 +-
 .../internal/cache/Bug41957DUnitTest.java       |    6 +-
 ...ssagesRegionCreationAndDestroyJUnitTest.java |   10 +-
 .../cache/ClientServerGetAllDUnitTest.java      |   40 +-
 .../cache/ClientServerTransactionDUnitTest.java |    6 +-
 .../cache/ConcurrentMapOpsDUnitTest.java        |    6 +-
 .../cache/DeltaPropagationDUnitTest.java        |    8 +-
 .../cache/DeltaPropagationStatsDUnitTest.java   |    2 +-
 .../internal/cache/DiskRegionJUnitTest.java     |    9 +-
 .../internal/cache/EventTrackerDUnitTest.java   |    8 +-
 .../internal/cache/GridAdvisorDUnitTest.java    |  106 +-
 .../internal/cache/HABug36773DUnitTest.java     |    8 +-
 .../HAOverflowMemObjectSizerDUnitTest.java      |   12 +-
 .../cache/OldValueImporterTestBase.java         |  165 ++
 .../cache/RemotePutReplyMessageJUnitTest.java   |   37 +
 .../control/RebalanceOperationDUnitTest.java    |    5 +-
 .../cache/execute/Bug51193DUnitTest.java        |    4 +-
 .../internal/cache/functions/TestFunction.java  |   10 +-
 .../cache/ha/Bug36853EventsExpiryDUnitTest.java |    8 +-
 .../internal/cache/ha/Bug48571DUnitTest.java    |    2 +-
 .../cache/ha/EventIdOptimizationDUnitTest.java  |   12 +-
 .../internal/cache/ha/FailoverDUnitTest.java    |   22 +-
 .../internal/cache/ha/HABugInPutDUnitTest.java  |    8 +-
 .../internal/cache/ha/HAClearDUnitTest.java     |   10 +-
 .../cache/ha/HAConflationDUnitTest.java         |    8 +-
 .../internal/cache/ha/HADuplicateDUnitTest.java |   10 +-
 .../cache/ha/HAEventIdPropagationDUnitTest.java |   10 +-
 .../internal/cache/ha/HAGIIDUnitTest.java       |   14 +-
 .../cache/ha/HARQueueNewImplDUnitTest.java      |   66 +-
 .../cache/ha/HASlowReceiverDUnitTest.java       |   10 +-
 .../ha/OperationsPropagationDUnitTest.java      |    8 +-
 .../internal/cache/ha/PutAllDUnitTest.java      |   12 +-
 .../cache/ha/StatsBugDUnitDisabledTest.java     |   14 +-
 .../cache/partitioned/Bug43684DUnitTest.java    |    4 +-
 .../PutPutReplyMessageJUnitTest.java            |   38 +
 .../fixed/FixedPartitioningTestBase.java        |    6 +-
 .../tier/sockets/AcceptorImplJUnitTest.java     |   62 +-
 ...rdCompatibilityCommandDUnitDisabledTest.java |  235 ---
 ...CompatibilityHandshakeDUnitDisabledTest.java |  218 ---
 ...mpatibilityHigherVersionClientDUnitTest.java |    4 +-
 ...rdCompatibilityMessageDUnitDisabledTest.java |  299 ----
 .../BridgeServerMaxConnectionsJUnitTest.java    |  221 ---
 ...geServerSelectorMaxConnectionsJUnitTest.java |   19 -
 .../tier/sockets/BridgeWriterMiscDUnitTest.java | 1384 ---------------
 .../BridgeWriterMiscSelectorDUnitTest.java      |   27 -
 .../cache/tier/sockets/Bug36269DUnitTest.java   |    4 +-
 .../cache/tier/sockets/Bug36457DUnitTest.java   |   18 +-
 .../cache/tier/sockets/Bug36805DUnitTest.java   |    6 +-
 .../cache/tier/sockets/Bug36995DUnitTest.java   |   11 +-
 .../cache/tier/sockets/Bug37210DUnitTest.java   |   10 +-
 .../CacheServerMaxConnectionsJUnitTest.java     |  220 +++
 ...heServerSelectorMaxConnectionsJUnitTest.java |   19 +
 .../cache/tier/sockets/CacheServerTestUtil.java |   51 +-
 .../CacheServerTransactionsDUnitTest.java       |    4 +-
 .../tier/sockets/ClearPropagationDUnitTest.java |    4 +-
 .../tier/sockets/ClientConflationDUnitTest.java |   20 +-
 .../sockets/ClientHealthMonitorJUnitTest.java   |    6 +-
 .../sockets/ClientInterestNotifyDUnitTest.java  |   14 +-
 .../tier/sockets/ClientServerMiscDUnitTest.java | 1381 +++++++++++++++
 .../ClientServerMiscSelectorDUnitTest.java      |   27 +
 .../cache/tier/sockets/ConflationDUnitTest.java |   24 +-
 .../tier/sockets/ConnectionProxyJUnitTest.java  |  257 +--
 .../DataSerializerPropogationDUnitTest.java     |   36 +-
 .../DestroyEntryPropagationDUnitTest.java       |   12 +-
 .../DurableClientReconnectDUnitTest.java        |   10 +-
 .../sockets/DurableClientStatsDUnitTest.java    |   10 +-
 .../sockets/DurableRegistrationDUnitTest.java   |    8 +-
 .../sockets/DurableResponseMatrixDUnitTest.java |   10 +-
 .../sockets/EventIDVerificationDUnitTest.java   |   10 +-
 .../ForceInvalidateEvictionDUnitTest.java       |   10 +-
 .../cache/tier/sockets/HAInterestBaseTest.java  |   64 +-
 .../tier/sockets/HAInterestPart2DUnitTest.java  |   12 +-
 .../sockets/HAStartupAndFailoverDUnitTest.java  |   81 +-
 .../tier/sockets/InterestListDUnitTest.java     |    4 +-
 .../sockets/InterestListEndpointDUnitTest.java  |   18 +-
 .../sockets/InterestListFailoverDUnitTest.java  |    6 +-
 .../sockets/InterestListRecoveryDUnitTest.java  |   10 +-
 .../sockets/InterestResultPolicyDUnitTest.java  |    4 +-
 .../tier/sockets/RedundancyLevelJUnitTest.java  |   14 +-
 .../sockets/RedundancyLevelPart1DUnitTest.java  |    4 +-
 .../sockets/RedundancyLevelPart3DUnitTest.java  |    6 +-
 .../tier/sockets/RedundancyLevelTestBase.java   |   51 +-
 .../tier/sockets/RegionCloseDUnitTest.java      |   14 +-
 ...erInterestBeforeRegionCreationDUnitTest.java |    6 +-
 .../sockets/RegisterInterestKeysDUnitTest.java  |    4 +-
 .../sockets/ReliableMessagingDUnitTest.java     |   24 +-
 .../internal/cache/tier/sockets/TestPut.java    |   53 -
 .../sockets/UpdatePropagationDUnitTest.java     |   10 +-
 ...UpdatesFromNonInterestEndPointDUnitTest.java |    4 +-
 .../logging/LogServiceIntegrationJUnitTest.java |   98 +-
 .../internal/logging/LogServiceJUnitTest.java   |   58 +-
 .../LogServiceUserDirIntegrationJUnitTest.java  |   70 -
 .../logging/LoggingIntegrationTestSuite.java    |    6 +-
 .../internal/logging/LoggingUnitTestSuite.java  |   15 +-
 .../logging/MergeLogFilesJUnitTest.java         |    4 +-
 .../log4j/FastLoggerIntegrationJUnitTest.java   |    4 +-
 .../FastLoggerWithDefaultConfigJUnitTest.java   |    4 +-
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java |  101 ++
 ...ersalMembershipListenerAdapterDUnitTest.java |   32 +-
 .../gemfire/pdx/ByteSourceJUnitTest.java        |    4 +-
 .../pdx/PdxFormatterPutGetJUnitTest.java        |    4 +-
 .../test/java/dunit/DistributedTestCase.java    |    4 +-
 .../gemfire/codeAnalysis/excludedClasses.txt    |    2 +-
 .../sanctionedDataSerializables.txt             |   14 +-
 .../codeAnalysis/sanctionedSerializables.txt    |   10 -
 .../tier/sockets/RedundancyLevelJUnitTest.xml   |   21 +-
 297 files changed, 9447 insertions(+), 15371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/build.gradle
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-assembly/build.gradle
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/DataSerializer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueManagerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java
index f15ac3f,a50e594..d07cd8d
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java
@@@ -8,19 -8,18 +8,18 @@@
  
  package com.gemstone.gemfire.distributed.internal;
  
 -import java.util.concurrent.ThreadPoolExecutor;
 +import java.util.List;
 +import java.util.concurrent.BlockingQueue;
  import java.util.concurrent.RejectedExecutionException;
  import java.util.concurrent.RejectedExecutionHandler;
 -import java.util.concurrent.BlockingQueue;
  import java.util.concurrent.SynchronousQueue;
 -import java.util.concurrent.TimeUnit;
  import java.util.concurrent.ThreadFactory;
 +import java.util.concurrent.ThreadPoolExecutor;
 +import java.util.concurrent.TimeUnit;
 +
  import com.gemstone.gemfire.SystemFailure;
  import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
- import com.gemstone.java.util.concurrent.SynchronousQueueNoSpin;
  
 -import java.util.List;
 -
  /**
   * A ThreadPoolExecutor with stat support.
   * @author darrel
@@@ -55,10 -54,10 +54,10 @@@ public class PooledExecutorWithDMStats 
    private Thread bufferConsumer;
    
    private static BlockingQueue<Runnable> initQ(BlockingQueue<Runnable> q) {
-     if (q instanceof SynchronousQueue || q instanceof SynchronousQueueNoSpin) {
+     if (q instanceof SynchronousQueue) {
        return q;
      } else {
 -      return new SynchronousQueue<Runnable>();
 +      return new SynchronousQueue/*NoSpin*/<Runnable>();
      }
    }
  

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/doc-files/properties.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
index 0b44cf4,b1464b9..50fec5d
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
@@@ -77,12 -77,10 +77,12 @@@ import com.gemstone.gemfire.internal.So
  import com.gemstone.gemfire.internal.SocketUtils;
  import com.gemstone.gemfire.internal.SystemTimer;
  import com.gemstone.gemfire.internal.Version;
 +import com.gemstone.gemfire.internal.VersionedDataInputStream;
 +import com.gemstone.gemfire.internal.VersionedDataOutputStream;
- import com.gemstone.gemfire.internal.cache.BridgeObserver;
- import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
- import com.gemstone.gemfire.internal.cache.BridgeRegionEventImpl;
- import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+ import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+ import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+ import com.gemstone.gemfire.internal.cache.ClientRegionEventImpl;
+ import com.gemstone.gemfire.internal.cache.CacheServerImpl;
  import com.gemstone.gemfire.internal.cache.CacheClientStatus;
  import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor;
  import com.gemstone.gemfire.internal.cache.CachedDeserializable;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
----------------------------------------------------------------------


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/FailoverDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/FailoverDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/FailoverDUnitTest.java
index af4e64f..7c623d9 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/FailoverDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/FailoverDUnitTest.java
@@ -20,14 +20,14 @@ 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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ConflationDUnitTest;
 import com.gemstone.gemfire.cache.client.PoolManager;
@@ -98,7 +98,7 @@ public class FailoverDUnitTest extends DistributedTestCase
     registerInterestList();
     primary.invoke(FailoverDUnitTest.class, "put");
     verifyEntries();
-    setBridgeObserver();
+    setClientServerObserver();
     primary.invoke(FailoverDUnitTest.class, "stopServer");
     verifyEntriesAfterFailover();
   }
@@ -135,7 +135,7 @@ public class FailoverDUnitTest extends DistributedTestCase
 */
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
-    BridgeTestCase.configureConnectionPoolWithName(factory, hostName, new int[] {PORT1,PORT2}, true, -1, 2, null, "FailoverPool");
+    ClientServerTestCase.configureConnectionPoolWithName(factory, hostName, new int[] {PORT1,PORT2}, true, -1, 2, null, "FailoverPool");
     factory.setCacheListener(new CacheListenerAdapter() {
       public void afterUpdate(EntryEvent event)
       {
@@ -157,7 +157,7 @@ public class FailoverDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     cache.createRegion(regionName, attrs);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
     server1.start();
@@ -224,9 +224,9 @@ public class FailoverDUnitTest extends DistributedTestCase
   public static void stopServer()
   {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
           server.stop();
       }
     }
@@ -270,9 +270,9 @@ public class FailoverDUnitTest extends DistributedTestCase
     assertEquals("value-3", r.getEntry("key-3").getValue());
   }
 
-  public static void setBridgeObserver() {
+  public static void setClientServerObserver() {
     PoolImpl.BEFORE_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
         public void beforePrimaryIdentificationFromBackup() {
           primary.invoke(FailoverDUnitTest.class, "putDuringFailover");
           PoolImpl.BEFORE_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = false;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HABugInPutDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HABugInPutDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HABugInPutDUnitTest.java
index b6a831e..d92c00a 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HABugInPutDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HABugInPutDUnitTest.java
@@ -19,12 +19,12 @@ 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.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 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;
@@ -132,7 +132,7 @@ public class HABugInPutDUnitTest extends DistributedTestCase
     factory.setDataPolicy(DataPolicy.REPLICATE);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
-    BridgeServerImpl server = (BridgeServerImpl)cache.addBridgeServer();
+    CacheServerImpl server = (CacheServerImpl)cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -152,7 +152,7 @@ public class HABugInPutDUnitTest extends DistributedTestCase
     new HABugInPutDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
-    BridgeTestCase.configureConnectionPool(factory, hostName, new int[] {PORT1,PORT2}, true, -1, 2, null);
+    ClientServerTestCase.configureConnectionPool(factory, hostName, new int[] {PORT1,PORT2}, true, -1, 2, null);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
     Region region = cache.getRegion(Region.SEPARATOR + REGION_NAME);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAClearDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAClearDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAClearDUnitTest.java
index f9d563f..1e93273 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAClearDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAClearDUnitTest.java
@@ -19,11 +19,11 @@ import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionEvent;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 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.CacheObserverAdapter;
 import com.gemstone.gemfire.internal.cache.CacheObserverHolder;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
@@ -61,7 +61,7 @@ public class HAClearDUnitTest extends DistributedTestCase
 
   protected static Cache cache = null;
 
-  static BridgeServerImpl server = null;
+  static CacheServerImpl server = null;
 
   static final int NO_OF_PUTS = 100;
 
@@ -572,7 +572,7 @@ public class HAClearDUnitTest extends DistributedTestCase
     factory.setDataPolicy(DataPolicy.REPLICATE);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
-    server = (BridgeServerImpl)cache.addBridgeServer();
+    server = (CacheServerImpl)cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -603,7 +603,7 @@ public class HAClearDUnitTest extends DistributedTestCase
     new HAClearDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
-    BridgeTestCase.configureConnectionPool(factory, hostName, new int[] {PORT1,PORT2}, true, -1, 2, null);
+    ClientServerTestCase.configureConnectionPool(factory, hostName, new int[] {PORT1,PORT2}, true, -1, 2, null);
     if (isListenerAttached) {
       factory.setCacheListener(new CacheListenerAdapter() {
         public void afterRegionClear(RegionEvent event)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAConflationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAConflationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAConflationDUnitTest.java
index 04a2901..ffec42d 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAConflationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAConflationDUnitTest.java
@@ -21,13 +21,13 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionEvent;
 import com.gemstone.gemfire.cache.Scope;
-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.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 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.tier.sockets.ConflationDUnitTest;
 
 import dunit.Host;
@@ -311,7 +311,7 @@ public class HAConflationDUnitTest extends CacheTestCase
     props.setProperty(DistributionConfig.LOCATORS_NAME, "");
     new HAConflationDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
-    BridgeTestCase.configureConnectionPool(factory, host, new int[] { PORT1 }, true, -1, -1, null);
+    ClientServerTestCase.configureConnectionPool(factory, host, new int[] { PORT1 }, true, -1, -1, null);
     factory.setScope(Scope.DISTRIBUTED_ACK);
     factory.setEnableConflation(true);
     if (isListenerPresent.booleanValue() == true) {
@@ -346,7 +346,7 @@ public class HAConflationDUnitTest extends CacheTestCase
     }
     RegionAttributes attrs = factory.create();
     cache.createRegion(regionName, attrs);
-    BridgeServerImpl server = (BridgeServerImpl)cache.addBridgeServer();
+    CacheServerImpl server = (CacheServerImpl)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/ha/HADuplicateDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HADuplicateDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HADuplicateDUnitTest.java
index 17412eb..830decd 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HADuplicateDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HADuplicateDUnitTest.java
@@ -24,11 +24,11 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 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 dunit.DistributedTestCase;
 import dunit.Host;
@@ -67,7 +67,7 @@ public class HADuplicateDUnitTest extends DistributedTestCase
 
   static boolean isEventDuplicate = true;
 
-  static BridgeServerImpl server = null;
+  static CacheServerImpl server = null;
 
   static final int NO_OF_PUTS = 100;
 
@@ -240,7 +240,7 @@ public class HADuplicateDUnitTest extends DistributedTestCase
     factory.setDataPolicy(DataPolicy.REPLICATE);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
-    server = (BridgeServerImpl)cache.addBridgeServer();
+    server = (CacheServerImpl)cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -269,7 +269,7 @@ public class HADuplicateDUnitTest extends DistributedTestCase
     props.setProperty("locators", "");
     new HADuplicateDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
-    BridgeTestCase.configureConnectionPool(factory, hostName, new int[] {PORT1,PORT2}, true, -1, 2, null);
+    ClientServerTestCase.configureConnectionPool(factory, hostName, new int[] {PORT1,PORT2}, true, -1, 2, null);
     
     factory.setScope(Scope.DISTRIBUTED_ACK);
     CacheListener clientListener = new HAValidateDuplicateListener();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAEventIdPropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAEventIdPropagationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAEventIdPropagationDUnitTest.java
index 16ce1c7..06beb27 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAEventIdPropagationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAEventIdPropagationDUnitTest.java
@@ -28,11 +28,11 @@ import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.QueueStateImpl.SequenceIdAndExpirationObject;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 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.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.RegionEventImpl;
@@ -66,7 +66,7 @@ public class HAEventIdPropagationDUnitTest extends DistributedTestCase
   private static Cache cache = null;
 
   /** server * */
-  static BridgeServerImpl server = null;
+  static CacheServerImpl server = null;
 
   /** test constructor * */
   public HAEventIdPropagationDUnitTest(String name) {
@@ -129,7 +129,7 @@ public class HAEventIdPropagationDUnitTest extends DistributedTestCase
     factory.setCacheListener(clientListener);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
-    server = (BridgeServerImpl)cache.addBridgeServer();
+    server = (CacheServerImpl)cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -160,7 +160,7 @@ public class HAEventIdPropagationDUnitTest extends DistributedTestCase
     props.setProperty("locators", "");
     new HAEventIdPropagationDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
-    PoolImpl pi = (PoolImpl)BridgeTestCase.configureConnectionPool(factory, hostName, new int[] {PORT1}, true, -1, 2, null);
+    PoolImpl pi = (PoolImpl)ClientServerTestCase.configureConnectionPool(factory, hostName, new int[] {PORT1}, true, -1, 2, null);
     factory.setScope(Scope.DISTRIBUTED_ACK);
     CacheListener clientListener = new HAEventIdPropagationListenerForClient();
     factory.setCacheListener(clientListener);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java
index 3229bf7..7ff64db 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HAGIIDUnitTest.java
@@ -23,9 +23,9 @@ import com.gemstone.gemfire.cache.Operation;
 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.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -176,7 +176,7 @@ public class HAGIIDUnitTest extends DistributedTestCase
     props.setProperty(DistributionConfig.LOCATORS_NAME, "");
     new HAGIIDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
-    BridgeTestCase.configureConnectionPool(factory, host, new int[] {PORT1,PORT2}, true, -1, 2, null, 1000, -1, false, -1);
+    ClientServerTestCase.configureConnectionPool(factory, host, new int[] {PORT1,PORT2}, true, -1, 2, null, 1000, -1, false, -1);
     factory.setScope(Scope.DISTRIBUTED_ACK);
     factory.addCacheListener(HAGIIDUnitTest.checker);
     RegionAttributes attrs = factory.create();
@@ -192,7 +192,7 @@ public class HAGIIDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
     server1.start();
@@ -207,7 +207,7 @@ public class HAGIIDUnitTest extends DistributedTestCase
     factory.setDataPolicy(DataPolicy.REPLICATE);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port.intValue());
     server1.setNotifyBySubscription(true);
     server1.start();
@@ -244,9 +244,9 @@ public class HAGIIDUnitTest extends DistributedTestCase
   public static void stopServer()
   {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
           server.stop();
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQueueNewImplDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQueueNewImplDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQueueNewImplDUnitTest.java
index ae2519c..c46dd19 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQueueNewImplDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HARQueueNewImplDUnitTest.java
@@ -27,14 +27,14 @@ 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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 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.LocalRegion;
 import com.gemstone.gemfire.internal.cache.ha.HAContainerMap;
 import com.gemstone.gemfire.internal.cache.ha.HAContainerRegion;
@@ -165,7 +165,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
     logger = cache.getLogger();
 
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
     if (ePolicy != null) {
@@ -186,7 +186,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
   public static Integer createOneMoreBridgeServer(Boolean notifyBySubscription)
       throws Exception {
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(notifyBySubscription.booleanValue());
     server1.getClientSubscriptionConfig().setEvictionPolicy(
@@ -205,7 +205,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
     props.setProperty(DistributionConfig.LOCATORS_NAME, "");
     new HARQueueNewImplDUnitTest("temp").createCache(props);
     AttributesFactory factory = new AttributesFactory();
-    BridgeTestCase.configureConnectionPool(factory, host, port1
+    ClientServerTestCase.configureConnectionPool(factory, host, port1
         .intValue(), port2.intValue(), true, Integer.parseInt(rLevel), 2, null,
         1000, 250, false);
 
@@ -893,7 +893,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   private static void verifyNullCUMReference(Integer port) {
     Region r = cache.getRegion("/"
-        + BridgeServerImpl.generateNameForClientMsgsRegion(port.intValue()));
+        + CacheServerImpl.generateNameForClientMsgsRegion(port.intValue()));
     assertNotNull(r);
 
     Object[] arr = r.keySet().toArray();
@@ -905,7 +905,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   private static void verifyHaContainerDestroyed(Boolean isRegion, Integer port) {
     Map r = cache.getRegion("/"
-        + BridgeServerImpl.generateNameForClientMsgsRegion(port.intValue()));
+        + CacheServerImpl.generateNameForClientMsgsRegion(port.intValue()));
 
     if (isRegion.booleanValue()) {
       if (r != null) {
@@ -913,7 +913,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
       }
     }
     else {
-      r = ((BridgeServerImpl)cache.getBridgeServers().toArray()[0])
+      r = ((CacheServerImpl)cache.getCacheServers().toArray()[0])
           .getAcceptor().getCacheClientNotifier().getHaContainer();
       if (r != null) {
         assertTrue(r.isEmpty());
@@ -942,9 +942,9 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   private static void setHACapacity(Integer cap) {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         server.getClientSubscriptionConfig().setCapacity(cap.intValue());
       }
     }
@@ -955,9 +955,9 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   private static void stopOneBridgeServer(Integer port) {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         if (server.getPort() == port.intValue()) {
           server.stop();
         }
@@ -970,9 +970,9 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   public static void stopServer() {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         server.stop();
       }
     }
@@ -1053,7 +1053,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
   public static void verifyNullValuesInCMR(final Integer numOfEntries, 
       final Integer port,
       String[] keys) {
-    final Region msgsRegion = cache.getRegion(BridgeServerImpl
+    final Region msgsRegion = cache.getRegion(CacheServerImpl
         .generateNameForClientMsgsRegion(port.intValue()));
     WaitCriterion wc = new WaitCriterion() {
       String excuse;
@@ -1082,7 +1082,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   public static void makeValuesOfSomeKeysNullInClientMsgsRegion(Integer port,
       String[] keys) {
-    Region msgsRegion = cache.getRegion(BridgeServerImpl
+    Region msgsRegion = cache.getRegion(CacheServerImpl
         .generateNameForClientMsgsRegion(port.intValue()));
     assertNotNull(msgsRegion);
 
@@ -1104,7 +1104,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   public static void populateValuesOfSomeKeysInClientMsgsRegion(Integer port,
       String[] keys) {
-    Region msgsRegion = cache.getRegion(BridgeServerImpl
+    Region msgsRegion = cache.getRegion(CacheServerImpl
         .generateNameForClientMsgsRegion(port.intValue()));
     assertNotNull(msgsRegion);
 
@@ -1117,9 +1117,9 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   public static void startServer() {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         server.start();
       }
     }
@@ -1132,7 +1132,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
       Integer msgsRegionsize, Integer port) {
     try {
       // Get the clientMessagesRegion and check the size.
-      Region msgsRegion = cache.getRegion(BridgeServerImpl
+      Region msgsRegion = cache.getRegion(CacheServerImpl
           .generateNameForClientMsgsRegion(port.intValue()));
       Region region = cache.getRegion("/" + regionName);
       logger.fine("size<serverRegion, clientMsgsRegion>: " + region.size()
@@ -1173,9 +1173,9 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
             return false;
           }
           
-          Iterator iter = cache.getBridgeServers().iterator();
+          Iterator iter = cache.getCacheServers().iterator();
           if (iter.hasNext()) {
-            BridgeServerImpl server = (BridgeServerImpl)iter.next();
+            CacheServerImpl server = (CacheServerImpl)iter.next();
             Map msgsRegion = server.getAcceptor().getCacheClientNotifier()
             .getHaContainer();
             //Region msgsRegion = cache.getRegion(BridgeServerImpl
@@ -1215,11 +1215,11 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
                 ", actual = " + sz;
             return false;
           }
-          Iterator iter = cache.getBridgeServers().iterator();
+          Iterator iter = cache.getCacheServers().iterator();
           if (!iter.hasNext()) {
             return true;
           }
-          BridgeServerImpl server = (BridgeServerImpl)iter.next();
+          CacheServerImpl server = (CacheServerImpl)iter.next();
           sz = server.getAcceptor().getCacheClientNotifier().getHaContainer().size();
           if (sz != msgsRegionsize.intValue()) {
             excuse = "Expected msgsRegionsize = " + msgsRegionsize.intValue() +
@@ -1241,19 +1241,19 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   public static void verifyHaContainerType(Boolean isRegion, Integer port) {
     try {
-      Map haMap = cache.getRegion(BridgeServerImpl
+      Map haMap = cache.getRegion(CacheServerImpl
           .generateNameForClientMsgsRegion(port.intValue()));
       if (isRegion.booleanValue()) {
         assertNotNull(haMap);
         assertTrue(haMap instanceof LocalRegion);
-        haMap = ((BridgeServerImpl)cache.getBridgeServers().toArray()[0])
+        haMap = ((CacheServerImpl)cache.getCacheServers().toArray()[0])
             .getAcceptor().getCacheClientNotifier().getHaContainer();
         assertNotNull(haMap);
         assertTrue(haMap instanceof HAContainerRegion);
       }
       else {
         assertNull(haMap);
-        haMap = ((BridgeServerImpl)cache.getBridgeServers().toArray()[0])
+        haMap = ((CacheServerImpl)cache.getCacheServers().toArray()[0])
             .getAcceptor().getCacheClientNotifier().getHaContainer();
         assertNotNull(haMap);
         assertTrue(haMap instanceof HAContainerMap);
@@ -1267,7 +1267,7 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
 
   public static void verifyRootRegionsDoesNotReturnCMR(Integer port) {
     try {
-      String cmrName = BridgeServerImpl.generateNameForClientMsgsRegion(port
+      String cmrName = CacheServerImpl.generateNameForClientMsgsRegion(port
           .intValue());
       Map haMap = cache.getRegion(cmrName);
       assertNotNull(haMap);
@@ -1323,12 +1323,12 @@ public class HARQueueNewImplDUnitTest extends DistributedTestCase {
     try {
       Map haContainer = null;
       haContainer = cache.getRegion(Region.SEPARATOR
-          + BridgeServerImpl.generateNameForClientMsgsRegion(port.intValue()));
+          + CacheServerImpl.generateNameForClientMsgsRegion(port.intValue()));
       if (haContainer == null) {
-        Object[] servers = cache.getBridgeServers().toArray();
+        Object[] servers = cache.getCacheServers().toArray();
         for (int i = 0; i < servers.length; i++) {
-          if (port.intValue() == ((BridgeServerImpl)servers[i]).getPort()) {
-            haContainer = ((BridgeServerImpl)servers[i]).getAcceptor()
+          if (port.intValue() == ((CacheServerImpl)servers[i]).getPort()) {
+            haContainer = ((CacheServerImpl)servers[i]).getAcceptor()
                 .getCacheClientNotifier().getHaContainer();
             break;
           }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HASlowReceiverDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HASlowReceiverDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HASlowReceiverDUnitTest.java
index a3eb9e4..90b8425 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HASlowReceiverDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/HASlowReceiverDUnitTest.java
@@ -20,13 +20,13 @@ 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.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
 import dunit.DistributedTestCase;
 import dunit.Host;
@@ -121,7 +121,7 @@ public class HASlowReceiverDUnitTest extends DistributedTestCase {
     logger = cache.getLogger();
 
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
     server1.setMaximumMessageCount(200);
@@ -260,7 +260,7 @@ public class HASlowReceiverDUnitTest extends DistributedTestCase {
   public static void setBridgeObeserverForAfterQueueDestroyMessage()
       throws Exception {
     PoolImpl.AFTER_QUEUE_DESTROY_MESSAGE_FLAG = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       @Override
       public void afterQueueDestroyMessage() {       
         clientVM.invoke(HASlowReceiverDUnitTest.class, "checkRedundancyLevel",

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/OperationsPropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/OperationsPropagationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/OperationsPropagationDUnitTest.java
index 384cb2a..10ef9d0 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/OperationsPropagationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/OperationsPropagationDUnitTest.java
@@ -19,11 +19,11 @@ 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.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 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;
@@ -170,7 +170,7 @@ public class OperationsPropagationDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     region = cache.createRegion(REGION_NAME, attrs);
 
-    BridgeServerImpl server = (BridgeServerImpl)cache.addBridgeServer();
+    CacheServerImpl server = (CacheServerImpl)cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -205,7 +205,7 @@ public class OperationsPropagationDUnitTest extends DistributedTestCase
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
     
-    BridgeTestCase.configureConnectionPool(factory, host, PORT2,-1, true, -1, 2, null);
+    ClientServerTestCase.configureConnectionPool(factory, host, PORT2,-1, true, -1, 2, null);
     RegionAttributes attrs = factory.create();
     region = cache.createRegion(REGION_NAME, attrs);
     assertNotNull(region);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/PutAllDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/PutAllDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/PutAllDUnitTest.java
index 5407160..7cd2ebf 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/PutAllDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/PutAllDUnitTest.java
@@ -26,11 +26,11 @@ import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.QueueStateImpl.SequenceIdAndExpirationObject;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 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.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 
@@ -73,7 +73,7 @@ public class PutAllDUnitTest extends DistributedTestCase
   /** cache **/
   private static Cache cache = null;
   /** server **/
-  static BridgeServerImpl server = null;
+  static CacheServerImpl server = null;
 
   /** test constructor **/
   public PutAllDUnitTest(String name) {
@@ -148,7 +148,7 @@ public class PutAllDUnitTest extends DistributedTestCase
     factory.setCacheListener(clientListener);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
-    server = (BridgeServerImpl)cache.addBridgeServer();
+    server = (CacheServerImpl)cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -189,7 +189,7 @@ public class PutAllDUnitTest extends DistributedTestCase
     props.setProperty("connectionsPerServer", "2");
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
-    PoolImpl p  = (PoolImpl)BridgeTestCase.configureConnectionPool(factory, host, PORT1,-1, true, -1, 2, null);
+    PoolImpl p  = (PoolImpl)ClientServerTestCase.configureConnectionPool(factory, host, PORT1,-1, true, -1, 2, null);
     CacheListener clientListener = new HAEventIdPropagationListenerForClient2();
     factory.setCacheListener(clientListener);
     RegionAttributes attrs = factory.create();
@@ -220,7 +220,7 @@ public class PutAllDUnitTest extends DistributedTestCase
     props.setProperty("connectionsPerServer", "2");
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
-    PoolImpl p = (PoolImpl)BridgeTestCase.configureConnectionPool(factory, host, PORT1,-1, true, -1, 2, null);
+    PoolImpl p = (PoolImpl)ClientServerTestCase.configureConnectionPool(factory, host, PORT1,-1, true, -1, 2, null);
     CacheListener clientListener = new HAEventIdPropagationListenerForClient1();
     factory.setCacheListener(clientListener);
     RegionAttributes attrs = factory.create();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/StatsBugDUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/StatsBugDUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/StatsBugDUnitDisabledTest.java
index a12ee6d..ad5050a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/StatsBugDUnitDisabledTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/StatsBugDUnitDisabledTest.java
@@ -18,8 +18,8 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-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.internal.AvailablePort;
 
@@ -209,7 +209,7 @@ public class StatsBugDUnitDisabledTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
 
     cache.createRegion(REGION_NAME, attrs);
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
     server.setNotifyBySubscription(false);
@@ -236,7 +236,7 @@ public class StatsBugDUnitDisabledTest extends DistributedTestCase
     cache = test.createCache(createProperties1());
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
-    pool = (PoolImpl)BridgeTestCase.configureConnectionPool(factory, host, new int[] {port1.intValue(),port2.intValue()}, true, -1, 3, null);
+    pool = (PoolImpl)ClientServerTestCase.configureConnectionPool(factory, host, new int[] {port1.intValue(),port2.intValue()}, true, -1, 3, null);
     RegionAttributes attrs = factory.create();
     Region region = cache.createRegion(REGION_NAME, attrs);
     region.registerInterest("ALL_KEYS");
@@ -260,7 +260,7 @@ public class StatsBugDUnitDisabledTest extends DistributedTestCase
     cache = test.createCache(createProperties1());
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.DISTRIBUTED_ACK);
-    pool = (PoolImpl)BridgeTestCase.configureConnectionPool(factory, host, new int[] {port1.intValue(),port2.intValue()}, true, -1, 3, null);
+    pool = (PoolImpl)ClientServerTestCase.configureConnectionPool(factory, host, new int[] {port1.intValue(),port2.intValue()}, true, -1, 3, null);
     RegionAttributes attrs = factory.create();
     Region region = cache.createRegion(REGION_NAME, attrs);
     region.registerInterest("ALL_KEYS", false, false);
@@ -295,9 +295,9 @@ public class StatsBugDUnitDisabledTest extends DistributedTestCase
   public static void stopServer()
   {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         server.stop();
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/Bug43684DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/Bug43684DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/Bug43684DUnitTest.java
index 999c63b..b2c8256 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/Bug43684DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/Bug43684DUnitTest.java
@@ -21,7 +21,7 @@ import com.gemstone.gemfire.cache.client.ClientRegionFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.OSProcess;
-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.LocalRegion;
 import com.gemstone.gemfire.internal.cache.RegionEntry;
@@ -248,7 +248,7 @@ public class Bug43684DUnitTest extends DistributedTestCase {
       rf.setPartitionAttributes(new PartitionAttributesFactory().setTotalNumBuckets(numBuckets).create());
     }
     rf.create(REGION_NAME);
-    BridgeServerImpl server = (BridgeServerImpl)cache.addCacheServer();
+    CacheServerImpl server = (CacheServerImpl)cache.addCacheServer();
     server.setPort(AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET));
     server.start();
     return server.getPort();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java
index d2b134f..7d34359 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/fixed/FixedPartitioningTestBase.java
@@ -40,9 +40,9 @@ import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.FileUtil;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.FixedPartitionAttributesImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.HARegion;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java
index 91f75fc..b92ef5c 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java
@@ -24,7 +24,7 @@ import org.junit.experimental.categories.Category;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.CacheFactory;
 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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
@@ -72,13 +72,13 @@ public class AcceptorImplJUnitTest
           port1,
           null,
           false,
-          BridgeServer.DEFAULT_SOCKET_BUFFER_SIZE,
-          BridgeServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
+          CacheServer.DEFAULT_SOCKET_BUFFER_SIZE,
+          CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
           this.cache,
           AcceptorImpl.MINIMUM_MAX_CONNECTIONS - 1,
-          BridgeServer.DEFAULT_MAX_THREADS,
-          BridgeServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-          BridgeServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null, false, false, Collections.EMPTY_LIST,
+          CacheServer.DEFAULT_MAX_THREADS,
+          CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
+          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null, false, false, Collections.EMPTY_LIST,
           CacheServer.DEFAULT_TCP_NO_DELAY);
         fail("Expected an IllegalArgumentExcption due to max conns < min pool size");
       } catch (IllegalArgumentException expected) {
@@ -89,13 +89,13 @@ public class AcceptorImplJUnitTest
           port2,
           null,
           false,
-          BridgeServer.DEFAULT_SOCKET_BUFFER_SIZE,
-          BridgeServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
+          CacheServer.DEFAULT_SOCKET_BUFFER_SIZE,
+          CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
           this.cache,
           0,
-          BridgeServer.DEFAULT_MAX_THREADS,
-          BridgeServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-          BridgeServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null,false, false, Collections.EMPTY_LIST,
+          CacheServer.DEFAULT_MAX_THREADS,
+          CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
+          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null,false, false, Collections.EMPTY_LIST,
           CacheServer.DEFAULT_TCP_NO_DELAY);
         fail("Expected an IllegalArgumentExcption due to max conns of zero");
       } catch (IllegalArgumentException expected) {
@@ -106,25 +106,25 @@ public class AcceptorImplJUnitTest
           port1,
           null,
           false,
-          BridgeServer.DEFAULT_SOCKET_BUFFER_SIZE,
-          BridgeServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
+          CacheServer.DEFAULT_SOCKET_BUFFER_SIZE,
+          CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
           this.cache,
           AcceptorImpl.MINIMUM_MAX_CONNECTIONS,
-          BridgeServer.DEFAULT_MAX_THREADS,
-          BridgeServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-          BridgeServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null,false, false, Collections.EMPTY_LIST,
+          CacheServer.DEFAULT_MAX_THREADS,
+          CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
+          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null,false, false, Collections.EMPTY_LIST,
           CacheServer.DEFAULT_TCP_NO_DELAY);
         a2 = new AcceptorImpl(
           port1,
           null,
           false,
-          BridgeServer.DEFAULT_SOCKET_BUFFER_SIZE,
-          BridgeServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
+          CacheServer.DEFAULT_SOCKET_BUFFER_SIZE,
+          CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
           this.cache,
           AcceptorImpl.MINIMUM_MAX_CONNECTIONS,
-          BridgeServer.DEFAULT_MAX_THREADS,
-          BridgeServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-          BridgeServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null,false, false, Collections.EMPTY_LIST,
+          CacheServer.DEFAULT_MAX_THREADS,
+          CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
+          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null,false, false, Collections.EMPTY_LIST,
           CacheServer.DEFAULT_TCP_NO_DELAY);
         fail("Expecetd a BindException while attaching to the same port");
       } catch (BindException expected) {
@@ -134,13 +134,13 @@ public class AcceptorImplJUnitTest
         port2,
         null,
         false,
-        BridgeServer.DEFAULT_SOCKET_BUFFER_SIZE,
-        BridgeServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
+        CacheServer.DEFAULT_SOCKET_BUFFER_SIZE,
+        CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
         this.cache,
         AcceptorImpl.MINIMUM_MAX_CONNECTIONS,
-        BridgeServer.DEFAULT_MAX_THREADS,
-        BridgeServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-        BridgeServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null, false, false, Collections.EMPTY_LIST,
+        CacheServer.DEFAULT_MAX_THREADS,
+        CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
+        CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null, false, false, Collections.EMPTY_LIST,
         CacheServer.DEFAULT_TCP_NO_DELAY);
       assertEquals(port2, a3.getPort());
       InternalDistributedSystem isystem = (InternalDistributedSystem) this.cache.getDistributedSystem();
@@ -187,13 +187,13 @@ public class AcceptorImplJUnitTest
         port2,
         null,
         false,
-        BridgeServer.DEFAULT_SOCKET_BUFFER_SIZE,
-        BridgeServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
+        CacheServer.DEFAULT_SOCKET_BUFFER_SIZE,
+        CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS,
         this.cache,
         AcceptorImpl.MINIMUM_MAX_CONNECTIONS,
-        BridgeServer.DEFAULT_MAX_THREADS,
-        BridgeServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-        BridgeServer.DEFAULT_MESSAGE_TIME_TO_LIVE,null,null);
+        CacheServer.DEFAULT_MAX_THREADS,
+        CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
+        CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,null,null);
       ac.start();
       putMsg.setMessageType(MessageType.PUT);
       putMsg.setTransactionId(1);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityCommandDUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityCommandDUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityCommandDUnitDisabledTest.java
deleted file mode 100755
index 05792fd..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityCommandDUnitDisabledTest.java
+++ /dev/null
@@ -1,235 +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.tier.sockets;
-
-import java.util.Properties;
-import java.util.HashMap;
-import java.util.Map;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-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.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.internal.cache.tier.ConnectionProxy;
-import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.TestPut;
-
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.VM;
-
-/**
- * @author Pallavi
- * 
- * Test to verify that server serves different versioned clients with their
- * respective client-versions of command .
- */
-public class BackwardCompatibilityCommandDUnitDisabledTest extends DistributedTestCase {
-  /** the cache */
-  private static Cache cache = null;
-
-  private static VM server1 = null;
-
-  private static VM client1 = null;
-
-  private static VM client2 = null;
-
-  /** name of the test region */
-  private static final String REGION_NAME = "BackwardCompatibilityCommandDUnitTest_Region";
-
-  static int CLIENT_ACK_INTERVAL = 5000;
-
-  private static final String k1 = "k1";
-
-  private static final String k2 = "k2";
-
-  private static final String client_k1 = "client-k1";
-
-  private static final String client_k2 = "client-k2";
-
-  public static boolean TEST_PUT_COMMAND_INVOKED = false;
-
-  /** constructor */
-  public BackwardCompatibilityCommandDUnitDisabledTest(String name) {
-    super(name);
-  }
-
-  public void setUp() throws Exception {
-    super.setUp();
-    final Host host = Host.getHost(0);
-    server1 = host.getVM(0);
-    client1 = host.getVM(1);
-    client2 = host.getVM(2);
-  }
-
-  private void createCache(Properties props) throws Exception {
-    DistributedSystem ds = getSystem(props);
-    ds.disconnect();
-    ds = getSystem(props);
-    assertNotNull(ds);
-    cache = CacheFactory.create(ds);
-    assertNotNull(cache);
-  }
-
-  public static void createClientCache(String host, Integer port1)
-      throws Exception {
-    new BackwardCompatibilityCommandDUnitDisabledTest("temp");
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new BackwardCompatibilityCommandDUnitDisabledTest("temp").createCache(props);
-    PoolImpl p = (PoolImpl)PoolManager.createFactory().addServer(host,
-        port1.intValue()).setSubscriptionEnabled(false)
-        .setThreadLocalConnections(true).setMinConnections(1).setReadTimeout(
-            20000).setPingInterval(10000).setRetryAttempts(1)
-        .create("BackwardCompatibilityCommandDUnitTest");
-
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setPoolName(p.getName());
-
-    RegionAttributes attrs = factory.create();
-    Region region = cache.createRegion(REGION_NAME, attrs);
-    //region.registerInterest("ALL_KEYS");
-  }
-
-  public static Integer createServerCache(String hostName) throws Exception {
-    new BackwardCompatibilityCommandDUnitDisabledTest("temp")
-        .createCache(new Properties());
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setDataPolicy(DataPolicy.REPLICATE);
-    RegionAttributes attrs = factory.create();
-    cache.createRegion(REGION_NAME, attrs);
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
-    server1.setBindAddress(hostName);
-    server1.setPort(port);
-    server1.setNotifyBySubscription(true);
-    server1.start();
-    return new Integer(server1.getPort());
-  }
-
-  public void tearDown2() throws Exception {
-    super.tearDown2();
-    // close the clients first
-    client2.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class,
-        "unsetHandshakeVersionForTesting");
-    client1.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class, "closeCache");
-    client2.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class, "closeCache");
-    // then close the servers
-    server1.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class, "closeCache");
-  }
-
-  public static void closeCache() {
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache.getDistributedSystem().disconnect();
-    }
-  }
-
-  /*
-   * Test to verify that server serves different versioned clients with their
-   * respective client-versions of command .
-   */
-  public void testCommand() {
-    server1.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class,
-        "setTestCommands");
-    String serverHostName = getServerHostName(server1.getHost());
-    Integer port1 = ((Integer)server1.invoke(
-        BackwardCompatibilityCommandDUnitDisabledTest.class, "createServerCache",
-        new Object[] { serverHostName }));
-
-    client1.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class,
-        "createClientCache", new Object[] { serverHostName, port1 });
-
-    client2.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class,
-        "setHandshakeVersionForTesting");
-    client2.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class,
-        "createClientCache", new Object[] { serverHostName, port1 });
-
-    client1.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class, "put");
-    server1.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class,
-        "checkTestPutCommandNotInvoked");
-
-    client2.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class, "put");
-    server1.invoke(BackwardCompatibilityCommandDUnitDisabledTest.class,
-        "checkTestPutCommandInvoked");
-  }
-
-  /*
-   * Add test command to CommandInitializer.ALL_COMMANDS.
-   */
-  public static void setTestCommands() throws Exception {
-    getLogWriter().info("setTestCommands invoked");
-    Map testCommands = new HashMap();
-    testCommands.putAll((Map)CommandInitializer.ALL_COMMANDS
-        .get(AcceptorImpl.VERSION));
-    testCommands.put(new Integer(MessageType.PUT), new TestPut());
-    CommandInitializer.testSetCommands(testCommands);
-    getLogWriter().info("end of setTestCommands");
-  }
-
-  /*
-   * Prepare to write TEST_VERSION byte from client to server during handshake.
-   */
-  public static void setHandshakeVersionForTesting() throws Exception {
-    HandShake.setVersionForTesting(Version.TEST_VERSION.ordinal());
-  }
-
-  public static void put() {
-    try {
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-      assertNotNull(r1);
-
-      r1.put(k1, client_k1);
-      assertEquals(r1.getEntry(k1).getValue(), client_k1);
-      r1.put(k2, client_k2);
-      assertEquals(r1.getEntry(k2).getValue(), client_k2);
-    }
-    catch (Exception ex) {
-      fail("failed while put", ex);
-    }
-  }
-
-  /*
-   * Prepare to write revert back to original version from client to server
-   * during handshake.
-   */
-  public static void unsetHandshakeVersionForTesting() throws Exception {
-    HandShake.setVersionForTesting(ConnectionProxy.VERSION.ordinal());
-  }
-
-  /*
-   * Check that TestPut command did not get invoked at server.
-   */
-  public static void checkTestPutCommandNotInvoked() {
-    assertTrue("TestPut command got invoked for GFE57 versioned client",
-        !TEST_PUT_COMMAND_INVOKED);
-  }
-
-  /*
-   * Check whether TestPut command got invoked at server.
-   */
-  public static void checkTestPutCommandInvoked() {
-    assertTrue(
-        "TestPut command did not get invoked for higher versioned client",
-        TEST_PUT_COMMAND_INVOKED);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHandshakeDUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHandshakeDUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHandshakeDUnitDisabledTest.java
deleted file mode 100755
index 9190d80..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHandshakeDUnitDisabledTest.java
+++ /dev/null
@@ -1,218 +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.tier.sockets;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-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.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.tier.ConnectionProxy;
-import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.VM;
-
-/**
- * @author Pallavi
- * 
- * Test to verify handshake with equal or lower version clients. Both equal and
- * lower version clients should be able to connect to the server successfully.
- */
-public class BackwardCompatibilityHandshakeDUnitDisabledTest extends
-    DistributedTestCase {
-
-  /** the cache */
-  private static Cache cache = null;
-
-  private static VM server1 = null;
-
-  private static VM client1 = null;
-
-  private static VM client2 = null;
-
-  /** name of the test region */
-  private static final String REGION_NAME = "BackwardCompatibilityHandshakeDUnitTest_Region";
-
-  static int CLIENT_ACK_INTERVAL = 5000;
-
-  private static short clientVersionForTesting = Version.TEST_VERSION.ordinal();
-
-  private static short currentClientVersion = ConnectionProxy.VERSION.ordinal();
-
-  /** constructor */
-  public BackwardCompatibilityHandshakeDUnitDisabledTest(String name) {
-    super(name);
-  }
-
-  public void setUp() throws Exception {
-    super.setUp();
-    final Host host = Host.getHost(0);
-    server1 = host.getVM(0);
-    client1 = host.getVM(1);
-    client2 = host.getVM(2);
-
-  }
-
-  private void createCache(Properties props) throws Exception {
-    DistributedSystem ds = getSystem(props);
-    ds.disconnect();
-    ds = getSystem(props);
-    assertNotNull(ds);
-    cache = CacheFactory.create(ds);
-    assertNotNull(cache);
-  }
-
-  public static void createClientCache(String host, Integer port1)
-      throws Exception {
-    new BackwardCompatibilityHandshakeDUnitDisabledTest("temp");
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new BackwardCompatibilityHandshakeDUnitDisabledTest("temp").createCache(props);
-    PoolImpl p = (PoolImpl)PoolManager.createFactory().addServer(host,
-        port1.intValue()).setSubscriptionEnabled(true)
-        .setThreadLocalConnections(true).setMinConnections(1).setReadTimeout(
-            20000).setPingInterval(10000).setRetryAttempts(1)
-        .setSubscriptionAckInterval(CLIENT_ACK_INTERVAL).create(
-            "BackwardCompatibilityHandshakeDUnitTest");
-
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setPoolName(p.getName());
-
-    RegionAttributes attrs = factory.create();
-    Region region = cache.createRegion(REGION_NAME, attrs);
-    region.registerInterest("ALL_KEYS");
-  }
-
-  public static Integer createServerCache(String serverHostName)
-      throws Exception {
-    new BackwardCompatibilityHandshakeDUnitDisabledTest("temp")
-        .createCache(new Properties());
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setDataPolicy(DataPolicy.REPLICATE);
-    RegionAttributes attrs = factory.create();
-    cache.createRegion(REGION_NAME, attrs);
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
-    server1.setBindAddress(serverHostName);
-    server1.setPort(port);
-    server1.setNotifyBySubscription(true);
-    server1.start();
-    return new Integer(server1.getPort());
-  }
-
-  public void tearDown2() throws Exception {
-    super.tearDown2();
-    client2.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class,
-        "unsetHandshakeVersionForTesting");
-    client2.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class,
-        "unsetTestVersionAfterHandshake");
-    // close the clients first
-    client1.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class, "closeCache");
-    client2.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class, "closeCache");
-    // then close the servers
-    server1.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class, "closeCache");
-  }
-
-  public static void closeCache() {
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache.getDistributedSystem().disconnect();
-    }
-  }
-
-  /*
-   * Test to verify handshake with equal or lower version clients. Both equal
-   * and lower version clients should be able to connect to the server
-   * successfully.
-   */
-  public void testHandShake() {
-    server1.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class,
-        "setTestCommands");
-    String serverHostName = getServerHostName(server1.getHost());
-    Integer port1 = ((Integer)server1.invoke(
-        BackwardCompatibilityHandshakeDUnitDisabledTest.class, "createServerCache",
-        new Object[] { serverHostName }));
-
-    client1.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class,
-        "createClientCache", new Object[] { serverHostName, port1 });
-
-    client2.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class,
-        "setHandshakeVersionForTesting");
-    client2.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class,
-        "setTestVersionAfterHandshake");
-    client2.invoke(BackwardCompatibilityHandshakeDUnitDisabledTest.class,
-        "createClientCache", new Object[] { serverHostName, port1 });
-  }
-
-  /*
-   * Add test command to CommandInitializer.ALL_COMMANDS.
-   */
-  public static void setTestCommands() throws Exception {
-    getLogWriter().info("setTestCommands invoked");
-    Map testCommands = new HashMap();
-    testCommands.putAll((Map)CommandInitializer.ALL_COMMANDS
-        .get(Version.GFE_57));
-    CommandInitializer.testSetCommands(testCommands);
-    getLogWriter().info("end of setTestCommands");
-  }
-
-  /*
-   * Prepare to write TEST_VERSION byte from client to server during
-   * handshake.
-   */
-  public static void setHandshakeVersionForTesting() throws Exception {
-    HandShake.setVersionForTesting(clientVersionForTesting);
-  }
-
-  /*
-   * Prepare to test that Server detected and created ClientHandshake as
-   * requested.
-   */
-  public static void setTestVersionAfterHandshake() throws Exception {
-    ServerConnection.TEST_VERSION_AFTER_HANDSHAKE_FLAG = true;
-    ServerConnection.testVersionAfterHandshake = clientVersionForTesting;
-  }
-
-  /*
-   * Prepare to revert back version byte to current client version for
-   * client-to-server handshake.
-   */
-  public static void unsetHandshakeVersionForTesting() throws Exception {
-    HandShake.setVersionForTesting(currentClientVersion);
-  }
-
-  /*
-   * Prepare to revert back Server testing of ClientHandshake as requested.
-   */
-  public static void unsetTestVersionAfterHandshake() throws Exception {
-    ServerConnection.TEST_VERSION_AFTER_HANDSHAKE_FLAG = false;
-    ServerConnection.testVersionAfterHandshake = (currentClientVersion);
-  }
-  /*
-   * public static void destroyRegion() { try { Region r = cache.getRegion("/" +
-   * REGION_NAME); assertNotNull(r); r.destroyRegion(); } catch (Exception ex) {
-   * fail("failed while destroy region ", ex); } }
-   */
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
index 7979fe1..35010c0 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityHigherVersionClientDUnitTest.java
@@ -16,7 +16,7 @@ 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.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -116,7 +116,7 @@ public class BackwardCompatibilityHigherVersionClientDUnitTest extends
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
     server1.start();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityMessageDUnitDisabledTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityMessageDUnitDisabledTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityMessageDUnitDisabledTest.java
deleted file mode 100755
index d308b00..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BackwardCompatibilityMessageDUnitDisabledTest.java
+++ /dev/null
@@ -1,299 +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.tier.sockets;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-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.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.Assert;
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
-import com.gemstone.gemfire.internal.cache.tier.ConnectionProxy;
-import com.gemstone.gemfire.cache.Region.Entry;
-import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy;
-
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.VM;
-
-/**
- * @author Pallavi
- * 
- * Test to verify that server serves different versioned clients with their
- * respective client-versions of messages (after execution of a command) .
- */
-
-public class BackwardCompatibilityMessageDUnitDisabledTest extends DistributedTestCase {
-  /** the cache */
-  private static Cache cache = null;
-
-  private static VM server1 = null;
-
-  private static VM client1 = null;
-
-  private static VM client2 = null;
-
-  private static VM client3 = null;
-
-  /** name of the test region */
-  private static final String REGION_NAME = "BackwardCompatibilityMessageDUnitTest_Region";
-
-  static int CLIENT_ACK_INTERVAL = 5000;
-
-  private static final String k1 = "k1";
-
-  private static final String k2 = "k2";
-
-  private static final String client_k1 = "client-k1";
-
-  private static final String client_k2 = "client-k2";
-
-  /** constructor */
-  public BackwardCompatibilityMessageDUnitDisabledTest(String name) {
-    super(name);
-  }
-
-  public void setUp() throws Exception {
-    super.setUp();
-    final Host host = Host.getHost(0);
-    server1 = host.getVM(0);
-    client1 = host.getVM(1);
-    client2 = host.getVM(2);
-    client3 = host.getVM(3);
-  }
-
-  private void createCache(Properties props) throws Exception {
-    DistributedSystem ds = getSystem(props);
-    ds.disconnect();
-    ds = getSystem(props);
-    assertNotNull(ds);
-    cache = CacheFactory.create(ds);
-    assertNotNull(cache);
-  }
-
-  public static void createClientCache(String host, Integer port1)
-      throws Exception {
-    new BackwardCompatibilityMessageDUnitDisabledTest("temp");
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new BackwardCompatibilityMessageDUnitDisabledTest("temp").createCache(props);
-    PoolImpl p = (PoolImpl)PoolManager.createFactory().addServer(host,
-        port1.intValue()).setSubscriptionEnabled(true)
-        .setSubscriptionRedundancy(1).setThreadLocalConnections(true)
-        .setMinConnections(6).setReadTimeout(20000).setPingInterval(10000)
-        .setRetryAttempts(1).setSubscriptionAckInterval(CLIENT_ACK_INTERVAL)
-        .create("BackwardCompatibilityMessageDUnitTest");
-
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setPoolName(p.getName());
-
-    RegionAttributes attrs = factory.create();
-    Region region = cache.createRegion(REGION_NAME, attrs);
-    region.registerInterest("ALL_KEYS");
-
-  }
-
-  public static Integer createServerCache(String host) throws Exception {
-    new BackwardCompatibilityMessageDUnitDisabledTest("temp")
-        .createCache(new Properties());
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setDataPolicy(DataPolicy.REPLICATE);
-    RegionAttributes attrs = factory.create();
-    cache.createRegion(REGION_NAME, attrs);
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
-    server1.setBindAddress(host);
-    server1.setPort(port);
-    server1.setNotifyBySubscription(true);
-    server1.start();
-    return new Integer(server1.getPort());
-  }
-
-  public void tearDown2() throws Exception {
-    super.tearDown2();
-    client2.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "unsetHandshakeVersionForTesting");
-    client3.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "unsetHandshakeVersionForTesting");
-    server1.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "unsetBridgeObserverForAfterMessageCreation");
-    // close the clients first
-    client1.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class, "closeCache");
-    client2.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class, "closeCache");
-    client3.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class, "closeCache");
-    // then close the servers
-    server1.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class, "closeCache");
-  }
-
-  public static void closeCache() {
-    CacheClientProxy.AFTER_MESSAGE_CREATION_FLAG = false;
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache.getDistributedSystem().disconnect();
-    }
-  }
-
-  /**
-   * Verify that server serves different versioned clients with their respective
-   * client-versions of messages (after execution of a command) .
-   */
-  public void testMessage() throws Exception {
-    String serverHostName = getServerHostName(server1.getHost());
-    server1.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "setTestCommands");
-    Integer port1 = ((Integer)server1.invoke(
-        BackwardCompatibilityMessageDUnitDisabledTest.class, "createServerCache",
-        new Object[] { serverHostName }));
-
-    client1.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "createClientCache", new Object[] { serverHostName, port1 });
-
-    client2.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "setHandshakeVersionForTesting");
-    client2.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "createClientCache", new Object[] { serverHostName, port1 });
-
-    client3.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "setHandshakeVersionForTesting");
-    client3.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "createClientCache", new Object[] { serverHostName, port1 });
-
-    server1.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class,
-        "setBridgeObserverForAfterMessageCreation");
-    client2.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class, "put");
-    Thread.sleep(10 * 1000);
-
-    client1.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class, "checkCache");
-    client3.invoke(BackwardCompatibilityMessageDUnitDisabledTest.class, "checkCache");
-  }
-
-  /*
-   * Add commands for TEST_VERSION to CommandInitializer.ALL_COMMANDS.
-   */
-  public static void setTestCommands() throws Exception {
-    getLogWriter().info("setTestCommands invoked");
-    Map testCommands = new HashMap();
-    testCommands.putAll((Map)CommandInitializer.ALL_COMMANDS
-        .get(Version.GFE_57));
-    CommandInitializer.testSetCommands(testCommands);
-    getLogWriter().info("end of setTestCommands");
-  }
-
-  /*
-   * Prepare to write TEST_VERSION byte from client to server during handshake.
-   */
-  public static void setHandshakeVersionForTesting() throws Exception {
-    HandShake.setVersionForTesting(Version.TEST_VERSION.ordinal());
-  }
-
-  private static BridgeObserver origObserver;
-
-  /*
-   * Prepare to test that ClientMessage created at server is valid for current
-   * client.
-   */
-  public static void setBridgeObserverForAfterMessageCreation()
-      throws Exception {
-    CacheClientProxy.AFTER_MESSAGE_CREATION_FLAG = true;
-    origObserver = BridgeObserverHolder
-        .setInstance(new BridgeObserverAdapter() {
-          public void afterMessageCreation(Message msg) {
-            getLogWriter().info("afterMessageCreation invoked");
-            Assert.assertTrue((msg != null),
-                "Valid Message not created for current client");
-
-            getLogWriter().info("end of afterMessageCreation");
-          }
-        });
-  }
-
-  public static void put() {
-    try {
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-      assertNotNull(r1);
-
-      r1.put(k1, client_k1);
-      assertEquals(r1.getEntry(k1).getValue(), client_k1);
-      r1.put(k2, client_k2);
-      assertEquals(r1.getEntry(k2).getValue(), client_k2);
-    }
-    catch (Exception ex) {
-      fail("failed while put", ex);
-    }
-  }
-
-  public static void checkCache() {
-    try {
-     final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME);
-     assertNotNull(r1);
-     WaitCriterion ev = new WaitCriterion() {
-       public boolean done() {
-         Entry e = r1.getEntry(k1);
-         return e != null;
-       }
-       public String description() {
-         return null;
-       }
-     };
-     DistributedTestCase.waitForCriterion(ev, 120 * 1000, 200, true);
-     Entry e = r1.getEntry(k1);
-     assertEquals(e.getValue(), client_k1);
-     
-     ev = new WaitCriterion() {
-       public boolean done() {
-         Entry e2 = r1.getEntry(k2);
-         return e2 != null;
-       }
-       public String description() {
-         return null;
-       }
-     };
-     DistributedTestCase.waitForCriterion(ev, 120 * 1000, 200, true);
-     e = r1.getEntry(k2);
-     assertEquals(e.getValue(), client_k2);
-    }
-    catch (Exception ex) {
-      fail("failed while checkCache", ex);
-    }
-  }
-
-  /*
-   * Prepare to write revert back to original version from client to server
-   * during handshake.
-   */
-  public static void unsetHandshakeVersionForTesting() throws Exception {
-    HandShake.setVersionForTesting(ConnectionProxy.VERSION.ordinal());
-  }
-
-  /*
-   * Prepare to revert back testing ClientMessage created at server.
-   */
-  public static void unsetBridgeObserverForAfterMessageCreation()
-      throws Exception {
-    CacheClientProxy.AFTER_MESSAGE_CREATION_FLAG = false;
-  }
-}


[31/38] incubator-geode git commit: GEODE-377: Add merge task for merging Distributed and Integration test coverage for consumption by SonarQube

Posted by bs...@apache.org.
GEODE-377: Add merge task for merging Distributed and Integration test coverage
for consumption by SonarQube


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

Branch: refs/heads/feature/GEODE-77
Commit: 17fdf57fdfa80a909e3f47d46408abe1ee45022a
Parents: 6c09b04
Author: Mark Bretl <mb...@pivotal.io>
Authored: Wed Sep 30 15:25:13 2015 -0700
Committer: Mark Bretl <mb...@pivotal.io>
Committed: Tue Oct 6 15:14:17 2015 -0700

----------------------------------------------------------------------
 build.gradle | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/17fdf57f/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index eda7642..c1534de 100755
--- a/build.gradle
+++ b/build.gradle
@@ -181,9 +181,18 @@ subprojects {
       }
     }
 
+    task mergeIntegrationTestCoverage (type: JacocoMerge) {
+      description 'Merges Distributed and Integration test coverage results'
+
+      destinationFile = file("${buildDir}/jacoco/mergedIntegrationTestCoverage.exec")
+      executionData = fileTree(dir: 'build/jacoco', include: ['**/distributedTest.exec','**/integrationTest.exec'])
+
+    }
+
     jacocoTestReport {
       reports { 
         csv.enabled false
+        sourceSets project.sourceSets.main
         html.destination "${buildDir}/jacocoTestHtml"
       }
     }


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeWriter.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeWriter.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeWriter.java
deleted file mode 100755
index 1f8a09a..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeWriter.java
+++ /dev/null
@@ -1,795 +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.cache.util;
-
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import com.gemstone.gemfire.LicenseException;
-import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.AttributesMutator;
-import com.gemstone.gemfire.cache.CacheWriter;
-import com.gemstone.gemfire.cache.CacheWriterException;
-import com.gemstone.gemfire.cache.Declarable;
-import com.gemstone.gemfire.cache.EntryEvent;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionEvent;
-import com.gemstone.gemfire.cache.client.internal.BridgePoolImpl;
-import com.gemstone.gemfire.internal.cache.tier.ConnectionProxy;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-
-/**
- * A <code>CacheWriter</code> that writes data to one or more remote
- * <code>CacheServer</code> processes. This allows for a hierarchical caching
- * scheme in which one cache ('client' cache) delegates a request to another
- * cache ('server' cache).
- * 
- * 
- * When using the <code>BridgeWriter</code>, at least two GemFire Caches must
- * exist in a client/server mode (they should not be part of the same
- * distributed system).
- * 
- * The 'server' cache must be running a gemfire <code>CacheServer</code>
- * process, while the 'client' cache must have a <code>BridgeWriter</code>
- * installed in one or more of its <code>Regions</code>. If a
- * <code>BridgeWriter</code> is defined in a client <code>Region</code>,
- * there must also be a <code>Region</code> defined in the 'server' cache with
- * the same exact name.
- * 
- * <p>
- * 
- * The <code>BridgeWriter</code> performs <code>put()</code> operations on
- * the remote server cache, and does not provide the distribution behavior that
- * can be enabled by using a <code>DISTRIBUTED</code> or
- * <code>DISTRIBUTED_NO_ACK</code> <code>Region</code>. This mechanism is
- * designed as a more targeted alternative to netSearch, in which the 'client'
- * cache completely delegates the loading of the data to the 'server' cache if
- * it is not yet cached in the client. This directed behavior enables a remote
- * network <code>put()</code> operation to be performed much more efficiently
- * in a scenario where there is a hierarchical cache topology. Updates and
- * invalidation remain local, in fact the <code>Regions</code> that are used
- * for this loosely coupled cache may even be <code>LOCAL</code> in scope.
- * 
- * The <code>BridgeWriter</code> may be used to configure caches with
- * multi-layer hierarchies.
- * 
- * 
- * <p>
- * <b>Load Balancing: </b>
- * <p>
- * The <code>BridgeWriter</code> supports these load balancing mechanisms
- * (specified by the <code>LBPolicy</code> config attribute):
- * <p>
- * <ul>
- * <li><b>Sticky </b> <br>
- * In this mode, the client writer picks the first server from the list of
- * servers and establishes a connection to it. Once this connection has been
- * established, every request from that particular 'client' cache is sent on
- * that connection. If requests time out or produce exceptions, the
- * <code>BridgeWriter</code> picks another server and then sends further
- * requests to that server. This achieves a level of load balancing by
- * redirecting requests away from servers that produce timeouts.</li>
- * 
- * <li><b>RandomSticky </b> <br>
- * The behavior is the same as Sticky, however the initial assignment of the
- * connection is randomly selected from the list of servers.</li>
- * 
- * <li><b>RoundRobin </b> <br>
- * In this mode, the client establishes connections to all the servers in the
- * server list and then randomly picks a server for each given request. For the
- * next request, it picks the next server in the list.</li>
- * 
- * <li><b>Random </b>: <br>
- * In this mode, the edge establishes connections to all the servers in the
- * server list and then randomly picks a server for every request.</li>
- * </ul>
- * 
- * <p>
- * <b>Failover: </b>
- * <p>
- * 
- * If a remote server cache throws an exception or times out, the client will
- * retry based on the configured <code>retryCount</code> parameter. If the
- * <code>retryCount</code> is exceeded, the server in question will be added
- * to a failed server list, and the client will select another server to connect
- * to. The servers in the failed server list will be periodically pinged with an
- * intelligent ping that ensures cache health. If a server is determined to be
- * healthy again, it will be promoted back to the healthy server list. The time
- * period between failed server pings is configurable via the
- * <code>retryInterval</code> parameter.
- * 
- * <p>
- * <b>Configuration: </b>
- * <p>
- * The <code>BridgeWriter</code> is configurable declaratively or
- * programmatically. Declarative configuration is achieved through defining the
- * configuration parameters in a <code>cache.xml</code> file. Programmatic
- * configuration may be achieved by first instantiating a
- * <code>BridgeWriter</code> object and subsequently calling
- * {@link #init(Properties)}with a <code>Properties</code> object containing
- * each desired parameter and value.
- * <p>
- * <b>The supported parameters are: </b>
- * <p>
- * <ul>
- * <li><b>endpoints </b> (required) <br>
- * A comma delimited list of logical names, hostnames, and ports of 'server'
- * caches to connect to <br>
- * The endpoints parameter follows this syntax:
- * logicalName=host:port,logicalName2=host2:port2,.... <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;endpoints&quot;&gt;
- *   &lt;string&gt;MyPrimaryServer=hostsrv:40404,MySecondary=hostsrv2:40404&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>readTimeout </b> (optional: default 10000) <br>
- * A millisecond value representing the amount of time to wait for a response
- * from a cache server. <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;readTimeout&quot;&gt;
- *   &lt;string&gt;5000&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * 
- * <li><b>retryAttempts </b> (optional: default 5)<br>
- * The number of times to retry a request after timeout/exception. <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;retryAttempts&quot;&gt;
- *   &lt;string&gt;5&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>retryInterval </b> (optional: default 10000) <br>
- * A millisecond value representing the amount of time to wait between attempts
- * by the <code>ServerMonitor</code> to ping living servers to verify that
- * they are still alive and dead servers to verify that they are still dead.
- * <br>
- * Example:</li>
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;retryInterval&quot;&gt;
- *   &lt;string&gt;10000&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>LBPolicy </b> (optional: default "Sticky") <br>
- * A String value representing the load balancing policy to use. See above for
- * more details. <br>
- * Options are:
- * <ul>
- * <li>"Sticky"</li>
- * <li>"RandomSticky"</li>
- * <li>"RoundRobin"</li>
- * <li>"Random"</li>
- * </ul>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;LBPolicy&quot;&gt;
- *   &lt;string&gt;Sticky&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>connectionsPerServer </b> (optional: default 1)<br>
- * The number of initial connections created to each time it is determined to be
- * alive. The minimum of <code>0</code> causes no initial connections to be
- * created (they are only created on demand). <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;connectionsPerServer&quot;&gt;
- *   &lt;string&gt;10&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>establishCallbackConnection </b> (optional: default false) <br>
- * Instruct the server to make a connection back to this edge client through
- * which the client receives cache updates. <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;establishCallbackConnection&quot;&gt;
- *   &lt;string&gt;true&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * 
- * <li><b>redundancyLevel </b> (optional: default 0) <br>
- * The number of secondary servers set for backup to the primary server for the
- * high availability of client queue. <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;redundancyLevel&quot;&gt;
- *   &lt;string&gt;1&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * 
- * <li><b>socketBufferSize </b> (optional: default 32768) <br>
- * The size of the socket buffers in bytes. <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;socketBufferSize&quot;&gt;
- *   &lt;string&gt;32768&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * 
- * <li><b>messageTrackingTimeout </b> (optional: default 300000 milliseconds)
- * <br>
- * messageTrackingTimeout property specifies the time-to-live period, in
- * milliseconds, for entries in the client's message tracking list, to minimize
- * duplicate events. Entries that have not been modified for this amount of time
- * are expired from the list <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;messageTrackingTimeout&quot;&gt;
- *   &lt;string&gt;300000&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>clientAckInterval</b> (optional: default 500 milliseconds) <br>
- * Bridge client sends an acknowledgement to its primary server for the events
- * it has got after every ClientAckInterval time.Client will send an ack to the
- * primary server only when redundancy level is greater than 0 or -1.<br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;clientAckInterval&quot;&gt;
- *   &lt;string&gt;5000&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * </ul>
- * 
- * <p>
- * 
- * If you are using a <code>cache.xml</code> file to create a
- * <code>Region</code> declaratively, you can include the following to
- * associate a <code>BridgeWriter</code> with a <code>Region</code> (default
- * values shown for optional parameters):
- * 
- * <pre>
- * 
- * &lt;cache-writer&gt;
- *   &lt;classname&gt;com.gemstone.gemfire.cache.util.BridgeWriter&lt;/classname&gt;
- *   &lt;parameter name=&quot;endpoints&quot;&gt;
- *     &lt;string&gt;MyHost=ninja.gemstone.com:40404&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;readTimeout&quot;&gt;
- *     &lt;string&gt;10000&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;retryAttempts&quot;&gt;
- *     &lt;string&gt;5&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;retryInterval&quot;&gt;
- *     &lt;string&gt;10000&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;LBPolicy&quot;&gt;
- *     &lt;string&gt;Sticky&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;establishCallbackConnection&quot;&gt;
- *     &lt;string&gt;false&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;socketBufferSize&quot;&gt;
- *     &lt;string&gt;32768&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;messageTrackingTimeout&quot;&gt;
- *     &lt;string&gt;300000&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;/parameter&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;clientAckInterval&quot;&gt;
- *      &lt;string&gt;5000&lt;/string&gt;
- *    &lt;/parameter&gt;
- * &lt;/cache-writer&gt;
- * </pre>
- * 
- * @since 3.5
- * @author Barry Oglesby
- * @deprecated as of 5.7 use {@link com.gemstone.gemfire.cache.client pools} instead.
- */
-
-@Deprecated
-public class BridgeWriter implements CacheWriter, Declarable
-{
-  protected ConnectionProxy proxy = null; // package access for
-                                    // tests/com/gemstone/gemfire/cache/util/BridgeHelper
-
-  private Properties properties;
-
-  private volatile boolean isClosed = false;
-
-  private final AtomicInteger refCount = new AtomicInteger();
-
-  // all writers logic was moved to ConnectionProxyImpl
-  
-  /**
-   * Initializes the writer with supplied config parameters. If instantiating
-   * the writer programmatically, this method must be called with a
-   * <code>Properties</code> object that at a minimum contains the 'endpoints'
-   * parameter before the writer can be used. If init fails with a
-   * LicenseException, the resulting BridgeWriter will throw
-   * IllegalStateException until it is properly initialized.
-   * 
-   * @param p configuration data such as 'endpoint' definitions
-   * @throws IllegalStateException if the writer is already initialized
-   */
-  public void init(Properties p)
-  {
-    if (this.proxy != null) throw new IllegalStateException(LocalizedStrings.BridgeWriter_ALREADY_INITIALIZED.toLocalizedString());
-    this.properties = p;
-    if (Boolean.getBoolean("skipConnection")) {
-      // used by hydra when generating XML via RegionAttributesCreation
-      return;
-    }
-    this.proxy = BridgePoolImpl.create(properties, true/*useByBridgeWriter*/);
-  }
-
-  /**
-   * Initializes this writer from an existing <code>BridgeWriter</code>. This
-   * method reuses the existing <code>BridgeWriter</code>'s proxy.
-   * 
-   * @param bridgeWriter
-   *          The existing <code>BridgeWriter</code>
-   * @throws IllegalStateException if the writer is already initialized
-   * 
-   * @since 4.2
-   */
-  public void init(BridgeWriter bridgeWriter)
-  {
-    if (this.proxy != null) throw new IllegalStateException(LocalizedStrings.BridgeWriter_ALREADY_INITIALIZED.toLocalizedString());
-    ConnectionProxy p = bridgeWriter.proxy;
-    p.reuse();
-    this.proxy = p;
-  }
-
-  /**
-   * Ensure that the BridgeClient and BridgePoolImpl classes
-   * get loaded.
-   * 
-   * @see SystemFailure#loadEmergencyClasses()
-   */
-  public static void loadEmergencyClasses() {
-    BridgeClient.loadEmergencyClasses(); // make sure subclass is taken care of
-    BridgePoolImpl.loadEmergencyClasses();
-  }
-
-  // emergency logic was moved to ConnectionProxyImpl
-  
-  /**
-   * Called when a region using this <code>BridgeWriter</code> is destroyed,
-   * when the cache is closed, or when a callback is removed from a region using
-   * an {@link AttributesMutator}.
-   * 
-   * Closes connections to {@link BridgeServer BridgeServers}when all
-   * {@link Region Regions}are finished using this BridgeWriter,
-   * 
-   * 
-   * @see #attach(Region)
-   * @see #detach(Region)
-   */
-  public void close()
-  {
-    if (this.refCount.get() <= 0) {
-      this.isClosed = true;
-      this.proxy.close();
-    }
-  }
-
-  // handleMarker moved to ConnectionProxyImpl
-
-  /**
-   * Returns true if this <code>BridgeWriter</code> has been closed.
-   */
-  public boolean isClosed() {
-    return this.isClosed;
-  }
-
-  private void checkClosed() {
-    String reason = this.proxy.getCancelCriterion().cancelInProgress();
-    if(reason != null) {
-      throw new BridgeWriterException("The BridgeWriter has been closed: " + reason);
-    }
-    
-    if (this.isClosed) {
-      throw new BridgeWriterException(LocalizedStrings.BridgeWriter_THE_BRIDGEWRITER_HAS_BEEN_CLOSED.toLocalizedString());
-    }
-    if (this.proxy != null && !this.proxy.isOpen()) {
-      throw new BridgeWriterException(LocalizedStrings.BridgeWriter_THE_BRIDGEWRITER_HAS_BEEN_CLOSED.toLocalizedString());
-    }
-  }
-  
-  /**
-   * Notify the BridgeWriter that the given region is no longer relevant. This
-   * method is used internally during Region
-   * {@link Region#destroyRegion() destruction}and
-   * {@link Region#close() closure}. This method effects the behavor of
-   * {@link #close()}.
-   * 
-   * @see #attach(Region)
-   * @see #close()
-   * @param r
-   *          the Region which will no longer use this BridgeWriter
-   * @since 4.3
-   */
-  public void detach(Region r)
-  {
-    this.refCount.decrementAndGet();
-    if (r != null) {
-      this.proxy.detachRegion(r);
-    }
-//    close(); // only closes if refCount is zero
-  }
-
-  /**
-   * Returns the number of attaches that have not yet called detach.
-   * @since 5.7
-   */
-  public int getAttachCount() {
-    return this.refCount.get();
-  }
-  
-  /**
-   * For speed optimizations, a connection to a server may be assigned to the
-   * calling thread when the BridgeWriter is used to do an operation.
-   * When the application thread is done doing its work it can invoke
-   * the BridgeWriter release method to make the connection available
-   * to other application threads.
-   */
-  public void release()
-  {
-    proxy.release();
-  }
-
-  /**
-   * This method should be invoked when the BridgeWriter mechanism is to be shut
-   * down explicitly , outside of closing the cache.
-   */
-  public void terminate()
-  {
-    this.isClosed = true;
-    proxy.terminate();
-  }
-
-  // removed checkForTransaction
-
-  /**
-   * Called before an entry is updated. The entry update is initiated by a
-   * <code>put</code> or a <code>get</code> that causes the writer to update
-   * an existing entry. The entry previously existed in the cache where the
-   * operation was initiated, although the old value may have been null. The
-   * entry being updated may or may not exist in the local cache where the
-   * CacheWriter is installed.
-   * 
-   * @param event
-   *          an EntryEvent that provides information about the operation in
-   *          progress
-   * @throws CacheWriterException
-   *           if thrown will abort the operation in progress, and the exception
-   *           will be propagated back to caller that initiated the operation
-   * @see Region#put(Object, Object)
-   * @see Region#get(Object)
-   */
-  public void beforeUpdate(EntryEvent event) throws CacheWriterException
-  {
-    throw new IllegalStateException("this method should not be called"); 
-  }
-
-  /**
-   * Called before an entry is created. Entry creation is initiated by a
-   * <code>create</code>, a <code>put</code>, or a <code>get</code>.
-   * The <code>CacheWriter</code> can determine whether this value comes from
-   * a <code>get</code> or not from {@link EntryEvent#isLoad}. The entry
-   * being created may already exist in the local cache where this
-   * <code>CacheWriter</code> is installed, but it does not yet exist in the
-   * cache where the operation was initiated.
-   * 
-   * @param event
-   *          an EntryEvent that provides information about the operation in
-   *          progress
-   * @throws CacheWriterException
-   *           if thrown will abort the operation in progress, and the exception
-   *           will be propagated back to caller that initiated the operation
-   * @see Region#create(Object, Object)
-   * @see Region#put(Object, Object)
-   * @see Region#get(Object)
-   */
-  public void beforeCreate(EntryEvent event) throws CacheWriterException
-  {
-    throw new IllegalStateException("this method should not be called"); 
-  }
-
-  /**
-   * Called before an entry is destroyed. The entry being destroyed may or may
-   * not exist in the local cache where the CacheWriter is installed. This
-   * method is <em>not</em> called as a result of expiration or
-   * {@link Region#localDestroy(Object)}.
-   * 
-   * @param event
-   *          an EntryEvent that provides information about the operation in
-   *          progress
-   * @throws CacheWriterException
-   *           if thrown will abort the operation in progress, and the exception
-   *           will be propagated back to caller that initiated the operation
-   * 
-   * @see Region#destroy(Object)
-   */
-  public void beforeDestroy(EntryEvent event) throws CacheWriterException
-  {
-    throw new IllegalStateException("this method should not be called"); 
-  }
-
-  /**
-   * Called before a region is destroyed. The <code>CacheWriter</code> will
-   * not additionally be called for each entry that is destroyed in the region
-   * as a result of a region destroy. If the region's subregions have
-   * <code>CacheWriter</code> s installed, then they will be called for the
-   * cascading subregion destroys. This method is <em>not</em> called as a
-   * result of expiration or {@link Region#localDestroyRegion()}. However, the
-   * {@link #close}method is invoked regardless of whether a region is
-   * destroyed locally. A non-local region destroy results in an invocation of
-   * {@link #beforeRegionDestroy}followed by an invocation of {@link #close}.
-   * <p>
-   * WARNING: This method should not destroy or create any regions itself or a
-   * deadlock will occur.
-   * 
-   * @param event
-   *          a RegionEvent that provides information about the
-   * 
-   * @throws CacheWriterException
-   *           if thrown, will abort the operation in progress, and the
-   *           exception will be propagated back to the caller that initiated
-   *           the operation
-   * 
-   * @see Region#destroyRegion()
-   */
-  public void beforeRegionDestroy(RegionEvent event)
-      throws CacheWriterException
-  {
-    throw new IllegalStateException("this method should not be called"); 
-  }
-
-  
-  /**
-   * Called before a region is cleared. The <code>CacheWriter</code> will
-   * not additionally be called for each entry that is cleared in the region
-   * as a result of a region clear. If the region's subregions have
-   * <code>CacheWriter</code> s installed, then they will be called for the
-   * cascading subregion clears. This method is <em>not</em> called as a
-   * result of expiration or {@link Region#localDestroyRegion()}. However, the
-   * {@link #close}method is invoked regardless of whether a region is
-   * cleared locally. A non-local region clear results in an invocation of
-   * {@link #beforeRegionClear}followed by an invocation of {@link #close}.
-   * <p>
-   * WARNING: This method should not destroy or create or clear any regions itself or a
-   * deadlock will occur.
-   * 
-   * @param event
-   *          a RegionEvent that provides information about the
-   * 
-   * @throws CacheWriterException
-   *           if thrown, will abort the operation in progress, and the
-   *           exception will be propagated back to the caller that initiated
-   *           the operation
-   * 
-   */
-  
-  public void beforeRegionClear(RegionEvent event) throws CacheWriterException
-  {
-    throw new IllegalStateException("this method should not be called"); 
-  }
-
-  /**
-   * Return true if this writer has not been closed and it was configured to
-   * establish a callback connection.
-   * 
-   * @since 4.3
-   */
-  public boolean hasEstablishCallbackConnection()
-  {
-    if (this.isClosed) {
-      return false;
-    }
-    else {
-      return this.proxy.getEstablishCallbackConnection();
-    }
-  }
-
-  // removed unregisterInterest
-
-  // removed getInterestList
-
-  // removed getObjectFromPrimaryServer
-
-  // removed keySet
-
-  // removed containsKey
-
-  /** Returns the retry interval in use. Retry interval refers to the interval
-   *  at which dead servers are attempted to be reconnected.
-   *  Internal use only.
-   */
-  public int getRetryInterval()
-  {
-    return proxy.getRetryInterval();
-  }
-
-  /**
-   * Returns the read timeout being used to time out requests to the server
-   * Internal use only.
-   */
-  public int getReadTimeout()
-  {
-    return proxy.getReadTimeout();
-  }
-
-  /**
-   * Returns the number of times the bridge writer tries to write data on
-   * encountering certain types of exceptions. Internal use only
-   */
-  public int getRetryAttempts()
-  {
-    return this.proxy.getRetryAttempts();
-  }
-
-  /**
-   * Returns the load balancing policy being used by the bridge writer Internal
-   * use only
-   */
-  public String getLBPolicy()
-  {
-    return proxy.getLBPolicy();
-  }
-
-  /**
-   * Returns the properties that defined this <code>BridgeWriter</code>.
-   * 
-   * @return the properties that defined this <code>BridgeWriter</code>
-   * 
-   * @since 4.2
-   */
-  public Properties getProperties()
-  {
-    return this.properties;
-  }
-
-  
-  /**
-   * Add an <code>Endpoint</code> to the known <code>Endpoint</code>s.
-   * 
-   * @param name The name of the endpoint to add
-   * @param host The host name or ip address of the endpoint to add
-   * @param port The port of the endpoint to add
-   * 
-   * @throws EndpointExistsException if the <code>Endpoint</code> to be
-   * added already exists.
-   * 
-   * @since 5.0.2
-   */
-  public void addEndpoint(String name, String host, int port)
-  throws EndpointExistsException {
-    this.proxy.addEndpoint(name, host, port);
-  }
-
-  /**
-   * Remove an <code>Endpoint</code> from the dead <code>Endpoint</code>s.
-   * The specified <code>Endpoint</code> must be dead.
-   * 
-   * @param name The name of the endpoint to remove
-   * @param host The host name or ip address of the endpoint to remove
-   * @param port The port of the endpoint to remove
-   * 
-   * @throws EndpointDoesNotExistException if the <code>Endpoint</code> to be
-   * removed doesn't exist.
-   * 
-   * @throws EndpointInUseException if the <code>Endpoint</code> to be removed
-   * contains <code>Connection</code>s
-   * 
-   * @since 5.0.2
-   */
-  public void removeEndpoint(String name, String host, int port)
-  throws EndpointDoesNotExistException, EndpointInUseException {
-    this.proxy.removeEndpoint(name, host, port);
-  }
-
-  // removed handleException
-
-  // removed getExceptionMessage
-
-  /**
-   * Returns a brief description of this <code>BridgeWriter</code>
-   * 
-   * @since 4.0
-   */
-  @Override
-  public String toString()
-  {
-    return LocalizedStrings.BridgeWriter_BRIDGEWRITER_CONNECTED_TO_0.toLocalizedString(this.proxy);
-  }
-
-  /**
-   * Notify the BridgeWriter that the given Region will begin delivering events
-   * to this BridgeWriter. This method effects the behavior of {@link #close()}
-   * 
-   * This is called internally when the BridgeWriter is added to a Region via
-   * {@link AttributesFactory#setCacheWriter(CacheWriter)}}
-   * 
-   * @param r
-   *          the Region which will begin use this BridgeWriter.
-   * @since 4.3
-   * 
-   * @see #detach(Region)
-   * @see #close()
-   */
-  public void attach(Region r)
-  {
-    checkClosed();
-    this.refCount.incrementAndGet();
-  }
-
-  /**
-   * Returns the <code>ConnectionProxy</code> associated with this
-   * <code>BridgeWriter</code>.
-   * 
-   * For internal use only.
-   * 
-   * @return the <code>ConnectionProxy</code> associated with this
-   *         <code>BridgeWriter</code>
-   */
-  public Object/*ConnectionProxy*/ getConnectionProxy() {
-    return proxy;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeWriterException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeWriterException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeWriterException.java
deleted file mode 100755
index e481c65..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeWriterException.java
+++ /dev/null
@@ -1,48 +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.cache.util;
-
-import com.gemstone.gemfire.cache.CacheWriterException;
-
-/**
- * An exception that is thrown by a {@link BridgeWriter} when a
- * problem occurs when communicating with a bridge server.
- *
- * @author David Whitlock
- * @since 3.5.2
- * @deprecated as of 5.7 use {@link com.gemstone.gemfire.cache.client pools} instead.
- */
-@Deprecated
-public class BridgeWriterException extends CacheWriterException {
-private static final long serialVersionUID = -295001316745954159L;
-
-  /**
-   * Creates a new <code>BridgeWriterException</code> with the given
-   * message. 
-   */
-  public BridgeWriterException(String message) {
-    super(message);
-  }
-
-  /**
-   * Creates a new <code>BridgeWriterException</code> with the given
-   * message and cause.
-   */
-  public BridgeWriterException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  /**
-   * Creates a new <code>BridgeWriterException</code> with the given
-   * cause.
-   */
-  public BridgeWriterException(Throwable cause) {
-    super(cause);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointDoesNotExistException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointDoesNotExistException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointDoesNotExistException.java
deleted file mode 100755
index 4f5eabd..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointDoesNotExistException.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-/**
- * An <code>EndpointDoesNotExistException</code> indicates a client
- * <code>Endpoint</code> does not exist for the input name, host and
- * port.
- *
- * @author Barry Oglesby
- *
- * @since 5.0.2
- * @deprecated as of 5.7 use {@link com.gemstone.gemfire.cache.client pools} instead.
- */
-@Deprecated
-public class EndpointDoesNotExistException extends EndpointException {
-private static final long serialVersionUID = 1654241470788247283L;
-
-  /**
-   * Constructs a new <code>EndpointDoesNotExistException</code>.
-   * 
-   * @param name The name of the requested <code>Endpoint</code>
-   * @param host The host of the requested <code>Endpoint</code>
-   * @param port The port of the requested <code>Endpoint</code>
-   */
-  public EndpointDoesNotExistException(String name, String host, int port) {
-    super(name+"->"+host+":"+port);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointException.java
deleted file mode 100755
index 445a75f..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointException.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-/**
- * An <code>EndpointException</code> is a generic exception that indicates
- * a client <code>Endpoint</code> exception has occurred. All other
- * <code>Endpoint</code> exceptions are subclasses of this class. Since
- * this class is abstract, only subclasses are instantiated.
- *
- * @author Barry Oglesby
- *
- * @since 5.0.2
- * @deprecated as of 5.7 use {@link com.gemstone.gemfire.cache.client pools} instead.
- */
-@Deprecated
-public abstract class EndpointException extends Exception {
-  
-  /** Constructs a new <code>EndpointException</code>. */
-  public EndpointException() {
-    super();
-  }
-
-  /** Constructs a new <code>EndpointException</code> with a message string. */
-  public EndpointException(String s) {
-    super(s);
-  }
-
-  /** Constructs a <code>EndpointException</code> with a message string and
-   * a base exception
-   */
-  public EndpointException(String s, Throwable cause) {
-    super(s, cause);
-  }
-
-  /** Constructs a <code>EndpointException</code> with a cause */
-  public EndpointException(Throwable cause) {
-    super(cause);
-  }
-
-  @Override
-  public String toString() {
-    String result = super.toString();
-    Throwable cause = getCause();
-    if (cause != null) {
-      String causeStr = cause.toString();
-      final String glue = ", caused by ";
-      StringBuffer sb = new StringBuffer(result.length() + causeStr.length() + glue.length());
-      sb.append(result)
-        .append(glue)
-        .append(causeStr);
-      result = sb.toString();
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointExistsException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointExistsException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointExistsException.java
deleted file mode 100755
index 3db6453..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointExistsException.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-
-/**
- * An <code>EndpointExistsException</code> indicates a client
- * <code>Endpoint</code> already exists.
- *
- * @author Barry Oglesby
- *
- * @since 5.0.2
- * @deprecated as of 5.7 use {@link com.gemstone.gemfire.cache.client pools} instead.
- */
-@Deprecated
-public class EndpointExistsException extends EndpointException {
-private static final long serialVersionUID = 950617116786308012L;
-
-  public EndpointExistsException(String msg) {
-    super(msg);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointInUseException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointInUseException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointInUseException.java
deleted file mode 100755
index 5c9093e..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/EndpointInUseException.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-
-/**
- * An <code>EndpointInUseException</code> indicates a client <code>Endpoint</code>
- * is in use (meaning that it contains one or more <code>Connection</code>s.
- *
- * @author Barry Oglesby
- *
- * @since 5.0.2
- * @deprecated as of 5.7 use {@link com.gemstone.gemfire.cache.client pools} instead.
- */
-@Deprecated
-public class EndpointInUseException extends EndpointException {
-private static final long serialVersionUID = -4087729485272321469L;
-
-  public EndpointInUseException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/IncompatibleVersionException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/IncompatibleVersionException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/IncompatibleVersionException.java
deleted file mode 100755
index fa72cd9..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/IncompatibleVersionException.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-import com.gemstone.gemfire.internal.Version;
-
-/**
- * An <code>Incompatible</code> indicates an unknown version.
- *
- * @author Barry Oglesby
- * @deprecated
- *
- * @since 5.6
- */
-public class IncompatibleVersionException extends VersionException {
-
-  private static final long serialVersionUID = 7008667865037538081L;
-
-  /**
-   * Constructs a new <code>IncompatibleVersionException</code>.
-   *
-   * @param clientVersion The client version
-   * @param serverVersion The server version
-   */
-  public IncompatibleVersionException(Object clientVersion,
-      Object serverVersion) {
-    // the arguments should be of class Version, but that's an
-    // internal class and this is an external class that shouldn't
-    // ref internals in method signatures
-    this("Client version " + clientVersion
-        + " is incompatible with server version " + serverVersion);
-  }
-
-  /**
-   * Constructs a new <code>IncompatibleVersionException</code>.
-   *
-   * @param message The exception message
-   */
-  public IncompatibleVersionException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/ServerRefusedConnectionException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/ServerRefusedConnectionException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/ServerRefusedConnectionException.java
deleted file mode 100755
index 2545e00..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/ServerRefusedConnectionException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-import com.gemstone.gemfire.distributed.DistributedMember;
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
-
-/**
- * A <code>ServerRefusedConnectionException</code> indicates a client attempted
- * to connect to a server, but the handshake was rejected.
- *
- * @author Barry Oglesby
- *
- * @since 5.5
- * @deprecated as of 5.7 use {@link com.gemstone.gemfire.cache.client.ServerRefusedConnectionException} from the <code>client</code> package instead.
- */
-@Deprecated
-@SuppressFBWarnings(value="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS", justification="class deprecated")
-public class ServerRefusedConnectionException extends com.gemstone.gemfire.cache.client.ServerRefusedConnectionException {
-private static final long serialVersionUID = -4996327025772566931L;
-  /**
-   * Constructs an instance of <code>ServerRefusedConnectionException</code> with the
-   * specified detail message.
-   * @param server the server that rejected the connection
-   * @param msg the detail message
-   */
-  public ServerRefusedConnectionException(DistributedMember server, String msg) {
-    super(server, msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/UniversalMembershipListenerAdapter.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/UniversalMembershipListenerAdapter.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/UniversalMembershipListenerAdapter.java
deleted file mode 100755
index 840ae09..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/UniversalMembershipListenerAdapter.java
+++ /dev/null
@@ -1,352 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-
-package com.gemstone.gemfire.cache.util;
-
-import com.gemstone.gemfire.admin.AdminDistributedSystem;
-import com.gemstone.gemfire.admin.SystemMembershipEvent;
-import com.gemstone.gemfire.admin.SystemMembershipListener;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.internal.Assert;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-
-import java.util.*;
-
-/**
- * <p>The <code>UniversalMembershipListenerAdapter</code> is a wrapper 
- * for {@link com.gemstone.gemfire.admin.SystemMembershipListener} and 
- * {@link BridgeMembershipListener}, providing a facade that makes both
- * appear to customer code as a single <code>SystemMembershipListener</code>
- * from the Admin API. This includes adapting 
- * <code>BridgeMembershipListener</code> events to appear as events for the 
- * <code>SystemMembershipListener</code>.</p>
- *
- * <p><code>UniversalMembershipListenerAdapter</code> implements
- * <code>SystemMembershipListener</code>, exposing the callbacks in that
- * interface as methods to be overridden by the customer.</p>
- *
- * <p>An internal implementation of <code>BridgeMembershipListener</code> is 
- * registered when this class is instantiated. This implementation creates a
- * {@link com.gemstone.gemfire.admin.SystemMembershipEvent} and calls the
- * corresponding <code>SystemMembershipListener</code> public methods on 
- * <code>UniversalMembershipListenerAdapter</code>. To the customer code, the
- * <code>BridgeMembershipEvent</code>s are wrapped to appear as 
- * <code>SystemMembershipEvent</code>s. In this way, both types of membership
- * events appear as <code>SystemMembershipEvent</code>s, allowing customer
- * code written using the Admin API to continue working by changing the
- * listener implementation to simply extend this class.</p>
- *
- * <p>Any BridgeServer using the <code>UniversalMembershipListenerAdapter</code>
- * will receive notifications of system membership changes and bridge
- * membership changes through a single listener.</p>
- *
- * <p>Any bridge client using the <code>UniversalMembershipListenerAdapter</code>
- * would receive notifications of bridge server connection changes. If that
- * bridge client also creates a connection to the GemFire {@link 
- * com.gemstone.gemfire.distributed.DistributedSystem}, then it will also
- * receive notifications of system membership changes.</p>
- *
- * <p>Subclasses of <code>UniversalMembershipListenerAdapter</code> may be
- * registered as a <code>SystemMembershipListener</code> using {@link 
- * com.gemstone.gemfire.admin.AdminDistributedSystem#addMembershipListener}.
- * It is best, however, to register the listener using {@link
- * #registerMembershipListener} since this allows the adapter to prevent
- * duplicate events for members that are both a system member and a bridge
- * member.</p>
- *
- * <p>Simply constructing the <code>UniversalMembershipListenerAdapter</code>
- * results in the underlying <code>BridgeMembershipListener</code> also being
- * registered.</p>
- *
- * <p>The following code illustrates how a BridgeServer application would use
- * <code>UniversalMembershipListenerAdapter</code>. The code in this example
- * assumes that the class MyMembershipListenerImpl extends 
- * <code>UniversalMembershipListenerAdapter</code>:
- * <pre><code>
- * public class MyMembershipListenerImpl extends UniversalMembershipListenerAdapter {
- *   public void memberCrashed(SystemMembershipEvent event) {
- *     // customer code
- *   }
- *   public void memberLeft(SystemMembershipEvent event) {
- *     // customer code
- *   }
- *   public void memberJoined(SystemMembershipEvent event) {
- *     // customer code
- *   }
- * }
- *
- * DistributedSystemConfig config = 
- *   AdminDistributedSystemFactory.defineDistributedSystem(myDS, null);
- * AdminDistributedSystem adminDS = 
- *   AdminDistributedSystemFactory.getDistributedSystem(config);
- * adminDS.connect();
- * MyMembershipListenerImpl myListener = new MyMembershipListenerImpl();
- * myListener.registerMembershipListener(adminDS);
- * </code></pre>
- * The callbacks on MyMembershipListenerImpl would then be
- * invoked for all <code>SystemMembershipEvent</code>s and 
- * <code>BridgeMembershipEvent</code>s. The latter will appear to be 
- * <code>SystemMembershipEvent</code>s.</p>
- *
- * <p>Similarly, the following code illustrates how a bridge client application
- * would use <code>UniversalMembershipListenerAdapter</code>, where 
- * MyMembershipListenerImpl is a subclass. Simply by constructing this subclass
- * of <code>UniversalMembershipListenerAdapter</code> it is registering itself
- * as a <code>BridgeMembershipListener</code>:
- * <pre><code>
- * new MyMembershipListenerImpl();
- * </code></pre>
- * A bridge client that also connects to the <code>DistributedSystem</code>
- * could register with the<code>AdminDistributedSystem</code> as shown 
- * above.</p>
- *
- * <p>It is recommended that subclasses register with the 
- * <code>AdminDistributedSystem</code> using {@link 
- * #registerMembershipListener}, as this will prevent duplicate events for
- * members that are both bridge members and system members. If duplicate
- * events are acceptable, you may register subclasses using {@link 
- * com.gemstone.gemfire.admin.AdminDistributedSystem#addMembershipListener 
- * AdminDistributedSystem#addMembershipListener}.</p>
- *
- * @author Kirk Lund
- * @since 4.2.1
- * @deprecated Use com.gemstone.gemfire.management.membership.UniversalMembershipListenerAdapter instead.
- */
-public abstract class UniversalMembershipListenerAdapter 
-implements SystemMembershipListener {
-  
-  /** 
-   * Default number of historical events to track in order to avoid duplicate
-   * events for members that are both bridge members and system members;
-   * value is 100.
-   */
-  public static final int DEFAULT_HISTORY_SIZE = 100;
-  
-//  private final Object[] eventHistory;
-//  private final boolean[] eventJoined;
-//  private boolean registered = false;
-  
-  protected final int historySize;
-  protected final LinkedList<String> eventHistory; // list of String memberIds
-  protected final Map<String,Boolean> eventJoined; // key: memberId, value: Boolean
-  
-  // TODO: perhaps ctor should require AdminDistributedSystem as arg?
-  
-  /** Constructs an instance of UniversalMembershipListenerAdapter. */
-  public UniversalMembershipListenerAdapter() {
-    this(DEFAULT_HISTORY_SIZE);
-  }
-  
-  /** 
-   * Constructs an instance of UniversalMembershipListenerAdapter.
-   * @param historySize number of historical events to track in order to avoid
-   * duplicate events for members that are both bridge members and system
-   * members; must a number between 10 and <code>Integer.MAX_INT</code>
-   * @throws IllegalArgumentException if historySizde is less than 10
-   */
-  public UniversalMembershipListenerAdapter(int historySize) {
-    if (historySize < 10) {
-      throw new IllegalArgumentException(LocalizedStrings.UniversalMembershipListenerAdapter_ARGUMENT_HISTORYSIZE_MUST_BE_BETWEEN_10_AND_INTEGERMAX_INT_0.toLocalizedString(Integer.valueOf(historySize)));
-    }
-    this.historySize = historySize;
-    this.eventHistory = new LinkedList<String>();
-    this.eventJoined = new HashMap<String,Boolean>();
-    BridgeMembership.registerBridgeMembershipListener(this.bridgeMembershipListener);
-  }
-  
-  /**
-   * Registers this adapter with the <code>AdminDistributedSystem</code>. 
-   * Registering in this way allows the adapter to ensure that callbacks will
-   * not be invoked twice for members that have a bridge connection and a
-   * system connection. If you register with {@link
-   * com.gemstone.gemfire.admin.AdminDistributedSystem#addMembershipListener}
-   * then duplicate events may occur for members that are both bridge members
-   * and system.
-   */
-  public void registerMembershipListener(AdminDistributedSystem admin) {
-    synchronized (this.eventHistory) {
-//      this.registered = true;
-      admin.addMembershipListener(this.systemMembershipListener);
-    }
-  }
-
-  /**
-   * Unregisters this adapter with the <code>AdminDistributedSystem</code>. 
-   * If registration is performed with {@link #registerMembershipListener}
-   * then this method must be used to successfuly unregister the adapter.
-   */
-  public void unregisterMembershipListener(AdminDistributedSystem admin) {
-    synchronized (this.eventHistory) {
-//      this.registered = false;
-      admin.removeMembershipListener(this.systemMembershipListener);
-    }
-    unregisterBridgeMembershipListener();
-  }
-  
-  /**
-   * Registers this adapter as a <code>BridgeMembershipListener</code>.
-   * Registration is automatic when constructing this adapter, so this call
-   * is no necessary unless it was previously unregistered by calling
-   * {@link #unregisterBridgeMembershipListener}.
-   */
-  public void registerBridgeMembershipListener() {
-    BridgeMembership.registerBridgeMembershipListener(this.bridgeMembershipListener);
-  }
-  
-  /**
-   * Unregisters this adapter as a <code>BridgeMembershipListener</code>.
-   * @see #registerBridgeMembershipListener
-   */
-  public void unregisterBridgeMembershipListener() {
-    BridgeMembership.unregisterBridgeMembershipListener(this.bridgeMembershipListener);
-  }
-  
-  /**
-   * Invoked when a member has joined the distributed system. Also invoked when
-   * a client has connected to this process or when this process has connected
-   * to a <code>BridgeServer</code>.
-   */
-  public void memberJoined(SystemMembershipEvent event) {}
-
-  /**
-   * Invoked when a member has gracefully left the distributed system. Also
-   * invoked when a client has gracefully disconnected from this process.
-   * or when this process has gracefully disconnected from a 
-   * <code>BridgeServer</code>.   */
-  public void memberLeft(SystemMembershipEvent event) {}
-
-  /**
-   * Invoked when a member has unexpectedly left the distributed system. Also
-   * invoked when a client has unexpectedly disconnected from this process
-   * or when this process has unexpectedly disconnected from a 
-   * <code>BridgeServer</code>.
-   */
-  public void memberCrashed(SystemMembershipEvent event) {}
-  
-  /** Adapts BridgeMembershipEvent to look like a SystemMembershipEvent */
-  public static class AdaptedMembershipEvent implements SystemMembershipEvent {
-    private final BridgeMembershipEvent event;
-
-    protected AdaptedMembershipEvent(BridgeMembershipEvent event) {
-      this.event = event;
-    }
-    /**
-     * Returns true if the member is a bridge client to a BridgeServer hosted
-     * by this process. Returns false if the member is a BridgeServer that this
-     * process is connected to.
-     */
-    public boolean isClient() {
-      return event.isClient();
-    }
-
-    public String getMemberId() {
-      return event.getMemberId();
-    }
-    
-    public DistributedMember getDistributedMember() {
-      return event.getMember();
-    }
-    
-    @Override
-    public boolean equals(Object other) {
-      if (other == this) return true;
-      if (other == null) return false;
-      if (!(other instanceof AdaptedMembershipEvent)) return  false;
-      final AdaptedMembershipEvent that = (AdaptedMembershipEvent) other;
-  
-      if (this.event != that.event &&
-          !(this.event != null &&
-          this.event.equals(that.event))) return false;
-  
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      return this.event.hashCode();
-    }
-
-    @Override
-    public String toString() {
-      final StringBuffer sb = new StringBuffer("[AdaptedMembershipEvent: ");
-      sb.append(this.event);
-      sb.append("]");
-      return sb.toString();
-    }
-  }
-
-  private final BridgeMembershipListener bridgeMembershipListener =
-  new BridgeMembershipListener() {
-    public void memberJoined(BridgeMembershipEvent event) {
-      systemMembershipListener.memberJoined(new AdaptedMembershipEvent(event));
-    }
-    public void memberLeft(BridgeMembershipEvent event) {
-      systemMembershipListener.memberLeft(new AdaptedMembershipEvent(event));
-    }
-    public void memberCrashed(BridgeMembershipEvent event) {
-      systemMembershipListener.memberCrashed(new AdaptedMembershipEvent(event));
-    }
-  };
-  
-  protected final SystemMembershipListener systemMembershipListener =
-  new SystemMembershipListener() {
-    public void memberJoined(SystemMembershipEvent event) {
-      if (!isDuplicate(event, true)) {
-        UniversalMembershipListenerAdapter.this.memberJoined(event);
-      }
-    }
-    public void memberLeft(SystemMembershipEvent event) {
-      if (!isDuplicate(event, false)) {
-        UniversalMembershipListenerAdapter.this.memberLeft(event);
-      }
-    }
-    public void memberCrashed(SystemMembershipEvent event) {
-      if (!isDuplicate(event, false)) {
-        UniversalMembershipListenerAdapter.this.memberCrashed(event);
-      }
-    }
-    protected boolean isDuplicate(SystemMembershipEvent event, boolean joined) {
-      synchronized (eventHistory) {
-        boolean duplicate = false;
-        String memberId = event.getMemberId();
-        
-        // find memberId in eventHistory...
-        int indexOf = eventHistory.indexOf(memberId);
-        if (indexOf > -1) {
-          // found an event for this member
-          if ((eventJoined.get(memberId)).booleanValue() == joined) {
-            // we already recorded a matching event for this member
-            duplicate = true;
-          }
-          else {
-            // remove the event from history and map... will be re-inserted
-            Assert.assertTrue(eventHistory.remove(memberId),
-              "Failed to replace entry in eventHistory for " + memberId);
-            Assert.assertTrue(eventJoined.remove(memberId) != null,
-              "Failed to replace entry in eventJoined for " + memberId);
-          }
-        }
-        
-        if (!duplicate) {
-          // add the event to the history and map
-          if (eventHistory.size() == historySize) {
-            // filled the eventHistory, so need to remove first entry
-            eventHistory.removeFirst();
-          }
-          eventHistory.addLast(memberId); // linked list
-          eventJoined.put(memberId, Boolean.valueOf(joined)); // boolean map
-          Assert.assertTrue(eventHistory.size() <= historySize,
-            "Attempted to grow eventHistory beyond maximum of " + historySize);
-        }
-        return duplicate;
-      } // sync
-    }
-  };
-  
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/UnknownVersionException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/UnknownVersionException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/UnknownVersionException.java
deleted file mode 100755
index 00743db..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/UnknownVersionException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-/**
- * An <code>UnknownVersionException</code> indicates an unknown version.
- *
- * @author Barry Oglesby
- * @deprecated
- *
- * @since 5.6
- */
-public class UnknownVersionException extends VersionException {
-
-  private static final long serialVersionUID = 7379530185697556990L;
-
-  /**
-   * Constructs a new <code>UnknownVersionException</code>.
-   * 
-   * @param versionOrdinal The ordinal of the requested <code>Version</code>
-   */
-  public UnknownVersionException(byte versionOrdinal) {
-    super(String.valueOf(versionOrdinal));
-  }
-  
-  /**
-   * Constructs a new <code>UnknownVersionException</code>.
-   * 
-   * @param message The exception message
-   */
-  public UnknownVersionException(String message) {
-    super(message);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/VersionException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/VersionException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/VersionException.java
deleted file mode 100755
index 200fac6..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/VersionException.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-import com.gemstone.gemfire.GemFireCheckedException;
-
-/**
- * An <code>VersionException</code> is an exception that indicates
- * a client / server version mismatch exception has occurred.
- *
- * @author Barry Oglesby
- * @deprecated Use {@link com.gemstone.gemfire.cache.VersionException} instead.
- *
- * @since 5.6
- */
-@Deprecated
-public abstract class VersionException extends GemFireCheckedException {
-
-  /** Constructs a new <code>VersionException</code>. */
-  public VersionException() {
-    super();
-  }
-
-  /** Constructs a new <code>VersionException</code> with a message string. */
-  public VersionException(String s) {
-    super(s);
-  }
-
-  /** Constructs a <code>VersionException</code> with a message string and
-   * a base exception
-   */
-  public VersionException(String s, Throwable cause) {
-    super(s, cause);
-  }
-
-  /** Constructs a <code>VersionException</code> with a cause */
-  public VersionException(Throwable cause) {
-    super(cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/package.html
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/package.html b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/package.html
index 296eba2..0bd948f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/package.html
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/package.html
@@ -12,8 +12,5 @@ Adapter classes are provided for convenience in implementing the
 CacheListener, RegionMembershipListener, RegionRoleListener, TransactionListener, and CacheWriter interfaces. These adapter
 classes simply implement the interfaces with empty methods so that subclasses
 only need implement the methods that are of interest.
-<p>
-Also provided are 
-BridgeLoader/BridgeWriter/BridgeServer, which are used for Hierarchical Caching.
   </BODY>
 </HTML>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
index a7cefbd..68efa34 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/ServerLauncher.java
@@ -40,7 +40,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 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.CacheConfig;
 import com.gemstone.gemfire.internal.cache.CacheServerLauncher;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -205,7 +205,7 @@ public final class ServerLauncher extends AbstractLauncher<String> {
   }
 
   private static Integer getDefaultServerPort() {
-    return Integer.getInteger(AbstractBridgeServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, CacheServer.DEFAULT_PORT);
+    return Integer.getInteger(AbstractCacheServer.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, CacheServer.DEFAULT_PORT);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
index e30b20b..76c2b92 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
@@ -74,14 +74,14 @@ import com.gemstone.gemfire.internal.StatisticsManager;
 import com.gemstone.gemfire.internal.StatisticsTypeFactoryImpl;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.admin.remote.DistributionLocatorId;
-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.EventID;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.execute.FunctionServiceStats;
 import com.gemstone.gemfire.internal.cache.execute.FunctionStats;
 import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
-import com.gemstone.gemfire.internal.cache.xmlcache.BridgeServerCreation;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheServerCreation;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogService;
@@ -2569,7 +2569,7 @@ public final class InternalDistributedSystem
     // the membership manager when forced-disconnect starts.  If we're
     // reconnecting for lost roles then this will be null
     String cacheXML = null;
-    List<BridgeServerCreation> cacheServerCreation = null;
+    List<CacheServerCreation> cacheServerCreation = null;
     
     GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
     boolean inhibitCacheForSQLFire = false;
@@ -2835,8 +2835,8 @@ public final class InternalDistributedSystem
             }
             cache = GemFireCacheImpl.create(this.reconnectDS, config);
             if (cacheServerCreation != null) {
-              for (BridgeServerCreation bridge: cacheServerCreation) {
-                BridgeServerImpl impl = (BridgeServerImpl)cache.addCacheServer();
+              for (CacheServerCreation bridge: cacheServerCreation) {
+                CacheServerImpl impl = (CacheServerImpl)cache.addCacheServer();
                 impl.configureFrom(bridge);
                 try {
                   if (!impl.isRunning()) {
@@ -2844,7 +2844,7 @@ public final class InternalDistributedSystem
                   }
                 } catch (IOException ex) {
                   throw new GemFireIOException(
-                      LocalizedStrings.CacheCreation_WHILE_STARTING_BRIDGE_SERVER_0
+                      LocalizedStrings.CacheCreation_WHILE_STARTING_CACHE_SERVER_0
                           .toLocalizedString(impl), ex);
                 }
               }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
index 33991bf..07cab82 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/ServerLocator.java
@@ -46,7 +46,7 @@ import com.gemstone.gemfire.distributed.internal.tcpserver.TcpHandler;
 import com.gemstone.gemfire.distributed.internal.tcpserver.TcpServer;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.cache.BridgeServerAdvisor.BridgeServerProfile;
+import com.gemstone.gemfire.internal.cache.CacheServerAdvisor.CacheServerProfile;
 import com.gemstone.gemfire.internal.cache.ControllerAdvisor;
 import com.gemstone.gemfire.internal.cache.ControllerAdvisor.ControllerProfile;
 import com.gemstone.gemfire.internal.cache.FindDurableQueueProcessor;
@@ -375,8 +375,8 @@ public class ServerLocator implements TcpHandler, DistributionAdvisee {
    * @param profile
    */
   public void profileCreated(Profile profile) {
-    if(profile instanceof BridgeServerProfile) {
-      BridgeServerProfile bp = (BridgeServerProfile) profile;
+    if(profile instanceof CacheServerProfile) {
+      CacheServerProfile bp = (CacheServerProfile) profile;
       ServerLocation location = buildServerLocation(bp);
       String[] groups = bp.getGroups();
       loadSnapshot.addServer(location, groups,
@@ -398,8 +398,8 @@ public class ServerLocator implements TcpHandler, DistributionAdvisee {
    * @param profile
    */
   public void profileRemoved(Profile profile) {
-    if(profile instanceof BridgeServerProfile) {
-      BridgeServerProfile bp = (BridgeServerProfile) profile;
+    if(profile instanceof CacheServerProfile) {
+      CacheServerProfile bp = (CacheServerProfile) profile;
       //InternalDistributedMember id = bp.getDistributedMember();
       ServerLocation location = buildServerLocation(bp);
       loadSnapshot.removeServer(location);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/jgroup/JGroupMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/jgroup/JGroupMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/jgroup/JGroupMembershipManager.java
index c589feb..c18d250 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/jgroup/JGroupMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/jgroup/JGroupMembershipManager.java
@@ -79,7 +79,7 @@ import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.internal.cache.DirectReplyMessage;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.xmlcache.BridgeServerCreation;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheServerCreation;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlGenerator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
@@ -2716,10 +2716,10 @@ public class JGroupMembershipManager implements MembershipManager
         }
       } else if (sharedConfigEnabled && !cache.getCacheServers().isEmpty()) {
         // we need to retain a cache-server description if this JVM was started by gfsh
-        List<BridgeServerCreation> list = new ArrayList<BridgeServerCreation>(cache.getCacheServers().size());
+        List<CacheServerCreation> list = new ArrayList<CacheServerCreation>(cache.getCacheServers().size());
         for (Iterator it = cache.getCacheServers().iterator(); it.hasNext(); ) {
           CacheServer cs = (CacheServer)it.next();
-          BridgeServerCreation bsc = new BridgeServerCreation(cache, cs);
+          CacheServerCreation bsc = new CacheServerCreation(cache, cs);
           list.add(bsc);
         }
         cache.getCacheConfig().setCacheServerCreation(list);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
index 3c33553..d556ce2 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
@@ -30,7 +30,7 @@ import com.gemstone.gemfire.admin.internal.PrepareBackupResponse;
 import com.gemstone.gemfire.admin.internal.SystemMemberCacheEventProcessor;
 import com.gemstone.gemfire.admin.jmx.internal.StatAlertNotification;
 import com.gemstone.gemfire.cache.InterestResultPolicy;
-import com.gemstone.gemfire.cache.client.internal.BridgeServerLoadMessage;
+import com.gemstone.gemfire.cache.client.internal.CacheServerLoadMessage;
 import com.gemstone.gemfire.cache.client.internal.locator.ClientConnectionRequest;
 import com.gemstone.gemfire.cache.client.internal.locator.ClientConnectionResponse;
 import com.gemstone.gemfire.cache.client.internal.locator.ClientReplacementRequest;
@@ -180,8 +180,8 @@ import com.gemstone.gemfire.internal.admin.remote.VersionInfoResponse;
 import com.gemstone.gemfire.internal.admin.statalerts.GaugeThresholdDecoratorImpl;
 import com.gemstone.gemfire.internal.admin.statalerts.NumberThresholdDecoratorImpl;
 import com.gemstone.gemfire.internal.cache.AddCacheServerProfileMessage;
-import com.gemstone.gemfire.internal.cache.BridgeRegionEventImpl;
-import com.gemstone.gemfire.internal.cache.BridgeServerAdvisor.BridgeServerProfile;
+import com.gemstone.gemfire.internal.cache.ClientRegionEventImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerAdvisor.CacheServerProfile;
 import com.gemstone.gemfire.internal.cache.BucketAdvisor;
 import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor;
 import com.gemstone.gemfire.internal.cache.CloseCacheMessage;
@@ -820,7 +820,7 @@ public final class DSFIDFactory implements DataSerializableFixedID {
     registerDSFID(GATEWAY_SENDER_PROFILE,
         GatewaySenderAdvisor.GatewaySenderProfile.class);
     registerDSFID(ROLE_EVENT, RoleEventImpl.class);
-    registerDSFID(BRIDGE_REGION_EVENT, BridgeRegionEventImpl.class);
+    registerDSFID(CLIENT_REGION_EVENT, ClientRegionEventImpl.class);
     registerDSFID(PR_INVALIDATE_MESSAGE, InvalidateMessage.class);
     registerDSFID(PR_INVALIDATE_REPLY_MESSAGE,
         InvalidateMessage.InvalidateReplyMessage.class);
@@ -860,8 +860,8 @@ public final class DSFIDFactory implements DataSerializableFixedID {
     registerDSFID(ALERTS_NOTIF_MESSAGE, AlertsNotificationMessage.class);
     registerDSFID(FIND_DURABLE_QUEUE, FindDurableQueueMessage.class);
     registerDSFID(FIND_DURABLE_QUEUE_REPLY, FindDurableQueueReply.class);
-    registerDSFID(BRIDGE_SERVER_LOAD_MESSAGE, BridgeServerLoadMessage.class);
-    registerDSFID(BRIDGE_SERVER_PROFILE, BridgeServerProfile.class);
+    registerDSFID(CACHE_SERVER_LOAD_MESSAGE, CacheServerLoadMessage.class);
+    registerDSFID(CACHE_SERVER_PROFILE, CacheServerProfile.class);
     registerDSFID(CONTROLLER_PROFILE, ControllerProfile.class);
     registerDSFID(DLOCK_QUERY_MESSAGE,
         DLockQueryProcessor.DLockQueryMessage.class);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
index a8a1715..84f038a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
@@ -135,7 +135,7 @@ public interface DataSerializableFixedID extends SerializationVersions {
   public static final byte PR_FUNCTION_REPLY_MESSAGE = -93;
 
   public static final byte PROFILES_REPLY_MESSAGE = -92;
-  public static final byte BRIDGE_SERVER_PROFILE = -91;
+  public static final byte CACHE_SERVER_PROFILE = -91;
   public static final byte CONTROLLER_PROFILE = -90;
 
   public static final byte CREATE_REGION_MESSAGE = -89;
@@ -341,12 +341,12 @@ public interface DataSerializableFixedID extends SerializationVersions {
   public static final byte PARTITION_PROFILE = 18;
 
   public static final byte ROLE_EVENT = 19;
-  public static final byte BRIDGE_REGION_EVENT = 20;
+  public static final byte CLIENT_REGION_EVENT = 20;
 
   public static final byte CONCURRENT_HASH_MAP = 21;
   public static final byte FIND_DURABLE_QUEUE = 22;
   public static final byte FIND_DURABLE_QUEUE_REPLY = 23;
-  public static final byte BRIDGE_SERVER_LOAD_MESSAGE = 24;
+  public static final byte CACHE_SERVER_LOAD_MESSAGE = 24;
 
   /** A header byte meaning that the next element in the stream is a
    * <code>ObjectPartList</code>.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/ClientStatsManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/ClientStatsManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/ClientStatsManager.java
index 7166024..b8ceb0b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/ClientStatsManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/ClientStatsManager.java
@@ -19,7 +19,6 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
 import com.gemstone.gemfire.cache.query.CqQuery;
-import com.gemstone.gemfire.cache.util.BridgeWriterException;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
@@ -88,12 +87,6 @@ public class ClientStatsManager {
     catch (DistributedSystemDisconnectedException e) {
       throw e;
     }
-    catch (BridgeWriterException bgx) {
-      pool.getCancelCriterion().checkCancelInProgress(bgx);
-      currentCache.getCancelCriterion().checkCancelInProgress(bgx);
-      //TODO: Need to analyze these exception scenarios.
-      logger.warning(LocalizedStrings.ClientStatsManager_FAILED_TO_SEND_CLIENT_HEALTH_STATS_TO_CACHESERVER, bgx);
-    }
     catch (CacheWriterException cwx) {
       pool.getCancelCriterion().checkCancelInProgress(cwx);
       currentCache.getCancelCriterion().checkCancelInProgress(cwx);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/GemFireVM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/GemFireVM.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/GemFireVM.java
index 289587a..1223152 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/GemFireVM.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/GemFireVM.java
@@ -314,7 +314,7 @@ public interface GemFireVM {
    *
    * @since 4.0
    */
-  public AdminBridgeServer addBridgeServer(CacheInfo cache)
+  public AdminBridgeServer addCacheServer(CacheInfo cache)
     throws AdminException;
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/BridgeServerResponse.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/BridgeServerResponse.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/BridgeServerResponse.java
index 62eae9b..6bfeef3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/BridgeServerResponse.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/BridgeServerResponse.java
@@ -12,7 +12,7 @@ import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 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 java.io.*;
 import java.util.*;
@@ -55,8 +55,8 @@ public final class BridgeServerResponse extends AdminResponse {
         int operation = request.getOperation();
         switch (operation) {
         case BridgeServerRequest.ADD_OPERATION: {
-          BridgeServerImpl bridge =
-            (BridgeServerImpl) cache.addBridgeServer();
+          CacheServerImpl bridge =
+            (CacheServerImpl) cache.addCacheServer();
           m.bridgeInfo = new RemoteBridgeServer(bridge);
           break;
         }
@@ -65,9 +65,9 @@ public final class BridgeServerResponse extends AdminResponse {
           int id = request.getBridgeId();
           // Note that since this is only an informational request
           // it is not necessary to synchronize on allBridgeServersLock
-          for (Iterator iter = cache.getBridgeServers().iterator();
+          for (Iterator iter = cache.getCacheServers().iterator();
                iter.hasNext(); ) {
-            BridgeServerImpl bridge = (BridgeServerImpl) iter.next();
+            CacheServerImpl bridge = (CacheServerImpl) iter.next();
             if (System.identityHashCode(bridge) == id) {
               m.bridgeInfo = new RemoteBridgeServer(bridge);
               break;
@@ -81,9 +81,9 @@ public final class BridgeServerResponse extends AdminResponse {
 
         case BridgeServerRequest.START_OPERATION: {
           RemoteBridgeServer config = request.getBridgeInfo();
-          for (Iterator iter = cache.getBridgeServers().iterator();
+          for (Iterator iter = cache.getCacheServers().iterator();
                iter.hasNext(); ) {
-            BridgeServerImpl bridge = (BridgeServerImpl) iter.next();
+            CacheServerImpl bridge = (CacheServerImpl) iter.next();
             if (System.identityHashCode(bridge) == config.getId()) {
               bridge.configureFrom(config);
               bridge.start();
@@ -99,9 +99,9 @@ public final class BridgeServerResponse extends AdminResponse {
 
         case BridgeServerRequest.STOP_OPERATION: {
           RemoteBridgeServer config = request.getBridgeInfo();
-          for (Iterator iter = cache.getBridgeServers().iterator();
+          for (Iterator iter = cache.getCacheServers().iterator();
                iter.hasNext(); ) {
-            BridgeServerImpl bridge = (BridgeServerImpl) iter.next();
+            CacheServerImpl bridge = (CacheServerImpl) iter.next();
             if (System.identityHashCode(bridge) == config.getId()) {
               bridge.stop();
               m.bridgeInfo = new RemoteBridgeServer(bridge);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/DurableClientInfoResponse.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/DurableClientInfoResponse.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/DurableClientInfoResponse.java
index dd5aeba..ecd4967 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/DurableClientInfoResponse.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/DurableClientInfoResponse.java
@@ -45,9 +45,9 @@ public class DurableClientInfoResponse extends AdminResponse
     try {
       GemFireCacheImpl c = (GemFireCacheImpl)CacheFactory.getInstanceCloseOk(dm
           .getSystem());
-      if (c.getBridgeServers().size() > 0) {
+      if (c.getCacheServers().size() > 0) {
 
-        BridgeServerImpl server = (BridgeServerImpl)c.getBridgeServers()
+        CacheServerImpl server = (CacheServerImpl)c.getCacheServers()
             .iterator().next();
         switch (request.action) {
         case DurableClientInfoRequest.HAS_DURABLE_CLIENT_REQUEST: {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java
index ec0bb34..cd40327 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteBridgeServer.java
@@ -27,8 +27,8 @@ import com.gemstone.gemfire.cache.server.ClientSubscriptionConfig;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.admin.AdminBridgeServer;
-import com.gemstone.gemfire.internal.cache.AbstractBridgeServer;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.AbstractCacheServer;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
 /**
@@ -40,7 +40,7 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
  * @since 4.0
  */
 public class RemoteBridgeServer
-  extends AbstractBridgeServer
+  extends AbstractCacheServer
   implements AdminBridgeServer, DataSerializable {
 
   private static final long serialVersionUID = 8417391824652384959L;
@@ -64,12 +64,12 @@ public class RemoteBridgeServer
    * <code>RemoteBridgeServer</code> from the contents of the given
    * <code>BridgeServerImpl</code>.
    */
-  RemoteBridgeServer(BridgeServerImpl impl) {
+  RemoteBridgeServer(CacheServerImpl impl) {
     super(null);
     this.port = impl.getPort();
     this.bindAddress = impl.getBindAddress();
     this.hostnameForClients = impl.getHostnameForClients();
-    if (BridgeServerImpl.ENABLE_NOTIFY_BY_SUBSCRIPTION_FALSE) {
+    if (CacheServerImpl.ENABLE_NOTIFY_BY_SUBSCRIPTION_FALSE) {
       this.notifyBySubscription = impl.getNotifyBySubscription();
     }
     this.socketBufferSize = impl.getSocketBufferSize();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteCacheInfo.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteCacheInfo.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteCacheInfo.java
index 689ef92..a3a283d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteCacheInfo.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteCacheInfo.java
@@ -12,7 +12,7 @@ import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.internal.admin.*;
 //import com.gemstone.gemfire.internal.*;
 import com.gemstone.gemfire.cache.*;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
@@ -79,11 +79,11 @@ public class RemoteCacheInfo implements CacheInfo, DataSerializable {
 
       // Note that since this is only a snapshot, so no synchronization
       // on allBridgeServersLock is needed.
-      Collection bridges = c.getBridgeServers();
+      Collection bridges = c.getCacheServers();
       this.bridgeServerIds = new int[bridges.size()];
       Iterator iter = bridges.iterator();
       for (int i = 0; iter.hasNext(); i++) {
-        BridgeServer bridge = (BridgeServer) iter.next();
+        CacheServer bridge = (CacheServer) iter.next();
         this.bridgeServerIds[i] = System.identityHashCode(bridge);
       }
 



[29/38] incubator-geode git commit: GEODE-243: make TXExpiryJUnitTest an integration test

Posted by bs...@apache.org.
GEODE-243: make TXExpiryJUnitTest an integration test


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

Branch: refs/heads/feature/GEODE-77
Commit: 48b0c961299435a7d9dd532ebe0a5ef20d9022c5
Parents: 8087818
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Oct 6 11:09:58 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Oct 6 11:11:27 2015 -0700

----------------------------------------------------------------------
 .../src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/48b0c961/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
index f8dffcc..2a0a581 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
@@ -42,7 +42,7 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.TXManagerImpl;
 import com.gemstone.gemfire.internal.cache.TXStateProxy;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 import dunit.DistributedTestCase;
 import dunit.DistributedTestCase.WaitCriterion;
@@ -54,7 +54,7 @@ import dunit.DistributedTestCase.WaitCriterion;
  * @since 4.0
  *
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class TXExpiryJUnitTest {
 
   protected GemFireCacheImpl cache;


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConnectionProxyJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConnectionProxyJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConnectionProxyJUnitTest.java
index db5a5f2..042917b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConnectionProxyJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConnectionProxyJUnitTest.java
@@ -31,11 +31,13 @@ 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.client.internal.BridgePoolImpl;
+import com.gemstone.gemfire.cache.client.PoolFactory;
+import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.PutOp;
 import com.gemstone.gemfire.cache.client.internal.QueueStateImpl.SequenceIdAndExpirationObject;
-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;
@@ -64,7 +66,7 @@ public class ConnectionProxyJUnitTest
 
   Cache cache;
 
-  BridgePoolImpl proxy = null;
+  PoolImpl proxy = null;
   
   SequenceIdAndExpirationObject seo = null;
   
@@ -100,7 +102,7 @@ public class ConnectionProxyJUnitTest
     
     this.system.disconnect();
     if (proxy != null)
-      proxy.close();
+      proxy.destroy();
   }
 
   /**
@@ -121,7 +123,7 @@ public class ConnectionProxyJUnitTest
     int port3 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     Region testRegion = null ;
     try {
-      BridgeServer server = this.cache.addBridgeServer();
+      CacheServer server = this.cache.addCacheServer();
       server.setMaximumTimeBetweenPings(10000);
       server.setPort(port3);
       server.start();
@@ -131,16 +133,18 @@ public class ConnectionProxyJUnitTest
       fail("Failed to create server");
     }
     try {
-      Properties props = new Properties();
-      props.setProperty("retryAttempts", "1");
-      props.setProperty("endpoints", "ep1=localhost:"+port3);
-      props.setProperty("establishCallbackConnection", "false");
-      props.setProperty("LBPolicy", "Sticky");
-      props.setProperty("readTimeout", "2000");
-      props.setProperty("socketBufferSize", "32768");
-      props.setProperty("retryInterval", "10000");
-      props.setProperty("connectionsPerServer", "2");
-      props.setProperty("redundancyLevel", "-1");
+      PoolFactory pf = PoolManager.createFactory();
+      pf.addServer("localhost", port3);
+      pf.setSubscriptionEnabled(false);
+      pf.setSubscriptionRedundancy(-1);
+      pf.setReadTimeout(2000);
+      pf.setThreadLocalConnections(true);
+      pf.setSocketBufferSize(32768);
+      pf.setRetryAttempts(1);
+      pf.setPingInterval(10000);
+      
+      proxy = (PoolImpl) pf.create("clientPool");
+
       AttributesFactory factory = new AttributesFactory();
       factory.setScope(Scope.DISTRIBUTED_ACK);
       factory.setCacheListener(new CacheListenerAdapter() {
@@ -156,8 +160,6 @@ public class ConnectionProxyJUnitTest
           }
         }
       });
-      proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
-
       RegionAttributes attrs = factory.create();
       testRegion = cache.createRegion("testregion", attrs);
 
@@ -200,17 +202,17 @@ public class ConnectionProxyJUnitTest
     
 //    final int maxWaitTime = 10000;    
     try {
-      Properties props = new Properties();
-      props.setProperty("retryAttempts", "1");
-      props.setProperty("endpoints", "ep1=localhost:"+port3);
-      props.setProperty("establishCallbackConnection", "false");
-      props.setProperty("LBPolicy", "Sticky");
-      props.setProperty("readTimeout", "2000");
-      props.setProperty("socketBufferSize", "32768");
-      props.setProperty("retryInterval", "500");
-      props.setProperty("connectionsPerServer", "1");
+      PoolFactory pf = PoolManager.createFactory();
+      pf.addServer("localhost", port3);
+      pf.setSubscriptionEnabled(false);
+      pf.setReadTimeout(2000);
+      pf.setThreadLocalConnections(true);
+      pf.setMinConnections(1);
+      pf.setSocketBufferSize(32768);
+      pf.setRetryAttempts(1);
+      pf.setPingInterval(500);
       
-      proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
+      proxy = (PoolImpl) pf.create("clientPool");
     }catch (Exception ex) {
       ex.printStackTrace();
       fail("Failed to initialize client");
@@ -230,10 +232,10 @@ public class ConnectionProxyJUnitTest
 //    long start = System.currentTimeMillis();
     assertEquals(0, proxy.getConnectedServerCount());
     //start the server
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
       server.setMaximumTimeBetweenPings(15000);
       server.setPort(port3);
       server.start();
@@ -272,16 +274,16 @@ public class ConnectionProxyJUnitTest
     
 //    final int maxWaitTime = 10000;    
     try {
-      Properties props = new Properties();
-      props.setProperty("retryAttempts", "1");
-      props.setProperty("endpoints", "ep1=localhost:"+port3);
-      props.setProperty("establishCallbackConnection", "false");
-      props.setProperty("LBPolicy", "Sticky");
-      props.setProperty("readTimeout", "2000");
-      props.setProperty("socketBufferSize", "32768");
-      props.setProperty("retryInterval", "500");
-      props.setProperty("connectionsPerServer", "1");
-      proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);      
+      PoolFactory pf = PoolManager.createFactory();
+      pf.addServer("localhost", port3);
+      pf.setSubscriptionEnabled(false);
+      pf.setReadTimeout(2000);
+      pf.setThreadLocalConnections(true);
+      pf.setMinConnections(1);
+      pf.setSocketBufferSize(32768);
+      pf.setRetryAttempts(1);
+      pf.setPingInterval(500);
+      proxy = (PoolImpl) pf.create("clientPool");
     }catch (Exception ex) {
       ex.printStackTrace();
       fail("Failed to initialize client");
@@ -291,10 +293,10 @@ public class ConnectionProxyJUnitTest
 //    long start = System.currentTimeMillis();
     assertEquals(0, proxy.getConnectedServerCount());
     //start the server
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
       server.setMaximumTimeBetweenPings(15000);
       server.setPort(port3);
       server.start();
@@ -324,10 +326,10 @@ public class ConnectionProxyJUnitTest
   public void testThreadIdToSequenceIdMapCreation()
   {
     int port3 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
       server.setMaximumTimeBetweenPings(10000);
       server.setPort(port3);
       server.start();
@@ -337,11 +339,11 @@ public class ConnectionProxyJUnitTest
       fail("Failed to create server");
     }
     try {
-      Properties props = new Properties();
-      props.setProperty("endpoints", "ep1=localhost:" + port3);
-      props.setProperty("establishCallbackConnection", "true");
-      props.setProperty("redundancyLevel", "-1");
-      proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
+      PoolFactory pf = PoolManager.createFactory();
+      pf.addServer("localhost", port3);
+      pf.setSubscriptionEnabled(true);
+      pf.setSubscriptionRedundancy(-1);
+      proxy = (PoolImpl) pf.create("clientPool");
       if (proxy.getThreadIdToSequenceIdMap() == null) {
         fail(" ThreadIdToSequenceIdMap is null. ");
       }
@@ -368,10 +370,10 @@ public class ConnectionProxyJUnitTest
   public void testThreadIdToSequenceIdMapExpiryPositive()
  {
    int port3 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
      server.setMaximumTimeBetweenPings(10000);
      server.setPort(port3);
      server.start();
@@ -381,14 +383,13 @@ public class ConnectionProxyJUnitTest
      fail("Failed to create server");
    }
    try {
-     Properties props = new Properties();
-     props.setProperty("endpoints", "ep1=localhost:"+port3);
-     props.setProperty("redundancyLevel", "-1");
-     props.setProperty("clientAckInterval", "2000");
-     props.setProperty("messageTrackingTimeout", "4000");
-     props.setProperty("establishCallbackConnection", "true");
-     
-     proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
+     PoolFactory pf = PoolManager.createFactory();
+     pf.addServer("localhost", port3);
+     pf.setSubscriptionEnabled(true);
+     pf.setSubscriptionRedundancy(-1);
+     pf.setSubscriptionMessageTrackingTimeout(4000);
+     pf.setSubscriptionAckInterval(2000);
+     proxy = (PoolImpl) pf.create("clientPool");
 
      EventID eid = new EventID(new byte[0],1,1);
      if(proxy.verifyIfDuplicate(eid)){
@@ -419,10 +420,10 @@ public class ConnectionProxyJUnitTest
   public void testThreadIdToSequenceIdMapExpiryNegative()
  {
    int port3 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
      server.setMaximumTimeBetweenPings(10000);     
      server.setPort(port3);
      server.start();
@@ -432,12 +433,13 @@ public class ConnectionProxyJUnitTest
      fail("Failed to create server");
    }
    try {
-     Properties props = new Properties();
-     props.setProperty("endpoints", "ep1=localhost:"+port3);
-     props.setProperty("establishCallbackConnection", "true");
-     props.setProperty("redundancyLevel", "-1");
-     props.setProperty("messageTrackingTimeout", "10000");
-     proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
+     PoolFactory pf = PoolManager.createFactory();
+     pf.addServer("localhost", port3);
+     pf.setSubscriptionEnabled(true);
+     pf.setSubscriptionRedundancy(-1);
+     pf.setSubscriptionMessageTrackingTimeout(10000);
+     
+     proxy = (PoolImpl) pf.create("clientPool");
 
      final EventID eid = new EventID(new byte[0],1,1);
      if(proxy.verifyIfDuplicate(eid)){
@@ -470,10 +472,10 @@ public class ConnectionProxyJUnitTest
   public void testThreadIdToSequenceIdMapConcurrency()
  {
    int port3 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
      server.setMaximumTimeBetweenPings(10000);
      server.setPort(port3);
      server.start();
@@ -483,14 +485,14 @@ public class ConnectionProxyJUnitTest
      fail("Failed to create server");
    }
    try {
-     Properties props = new Properties();
-     props.setProperty("endpoints", "ep1=localhost:"+port3);
-     props.setProperty("redundancyLevel", "-1");
-     props.setProperty("clientAckInterval", "2000");
-     props.setProperty("messageTrackingTimeout", "5000");
-     props.setProperty("establishCallbackConnection", "true");
+     PoolFactory pf = PoolManager.createFactory();
+     pf.addServer("localhost", port3);
+     pf.setSubscriptionEnabled(true);
+     pf.setSubscriptionRedundancy(-1);
+     pf.setSubscriptionMessageTrackingTimeout(5000);
+     pf.setSubscriptionAckInterval(2000);
+     proxy = (PoolImpl) pf.create("clientPool");
      
-     proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
      final int EVENT_ID_COUNT = 10000; // why 10,000?
      EventID[] eid = new EventID[EVENT_ID_COUNT];
      for (int i = 0; i < EVENT_ID_COUNT; i++) {
@@ -525,10 +527,10 @@ public class ConnectionProxyJUnitTest
   public void testDuplicateSeqIdLesserThanCurrentSeqIdBeingIgnored()
  {
    int port3 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
      server.setMaximumTimeBetweenPings(10000);
      server.setPort(port3);
      server.start();
@@ -538,12 +540,13 @@ public class ConnectionProxyJUnitTest
      fail("Failed to create server");
    }
    try {
-     Properties props = new Properties();
-     props.setProperty("endpoints", "ep1=localhost:"+port3);
-     props.setProperty("establishCallbackConnection", "true");
-     props.setProperty("redundancyLevel", "-1");
-     props.setProperty("messageTrackingTimeout", "100000");
-     proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
+     PoolFactory pf = PoolManager.createFactory();
+     pf.addServer("localhost", port3);
+     pf.setSubscriptionEnabled(true);
+     pf.setSubscriptionRedundancy(-1);
+     pf.setSubscriptionMessageTrackingTimeout(100000);
+     proxy = (PoolImpl) pf.create("clientPool");
+     
      EventID eid1 = new EventID(new byte[0],1,5);
      if(proxy.verifyIfDuplicate(eid1)){
        fail(" eid1 can never be duplicate, it is being created for the first time! ");
@@ -562,7 +565,7 @@ public class ConnectionProxyJUnitTest
      }
 
      assertTrue(!proxy.getThreadIdToSequenceIdMap().isEmpty());
-     proxy.close();
+     proxy.destroy();
    }
    catch (Exception ex) {
      ex.printStackTrace();
@@ -583,10 +586,10 @@ public class ConnectionProxyJUnitTest
   public void testCleanCloseOfThreadIdToSeqId()
  {
    int port3 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
      server.setMaximumTimeBetweenPings(10000);
      server.setPort(port3);
      server.start();
@@ -596,12 +599,13 @@ public class ConnectionProxyJUnitTest
      fail("Failed to create server");
    }
    try {
-     Properties props = new Properties();
-     props.setProperty("endpoints", "ep1=localhost:"+port3);
-     props.setProperty("establishCallbackConnection", "true");
-     props.setProperty("redundancyLevel", "-1");
-     props.setProperty("messageTrackingTimeout", "100000");
-     proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
+     PoolFactory pf = PoolManager.createFactory();
+     pf.addServer("localhost", port3);
+     pf.setSubscriptionEnabled(true);
+     pf.setSubscriptionRedundancy(-1);
+     pf.setSubscriptionMessageTrackingTimeout(100000);
+     proxy = (PoolImpl) pf.create("clientPool");
+     
      EventID eid1 = new EventID(new byte[0],1,2);
      if(proxy.verifyIfDuplicate(eid1)){
          fail(" eid can never be duplicate, it is being created for the first time! ");
@@ -640,10 +644,10 @@ public class ConnectionProxyJUnitTest
   public void testTwoClientsHavingDifferentThreadIdMaps()
  {
    int port3 = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
      server.setMaximumTimeBetweenPings(10000);
      server.setPort(port3);
      server.start();
@@ -653,22 +657,28 @@ public class ConnectionProxyJUnitTest
      fail("Failed to create server");
    }
    try {
-     Properties props = new Properties();
-     props.setProperty("endpoints", "ep1=localhost:"+port3);
-     props.setProperty("establishCallbackConnection", "true");
-     props.setProperty("redundancyLevel", "-1");
-     props.setProperty("messageTrackingTimeout", "100000");
-     BridgePoolImpl proxy1 = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
-     BridgePoolImpl proxy2 = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);
+     PoolFactory pf = PoolManager.createFactory();
+     pf.addServer("localhost", port3);
+     pf.setSubscriptionEnabled(true);
+     pf.setSubscriptionRedundancy(-1);
+     pf.setSubscriptionMessageTrackingTimeout(100000);
+     
+     PoolImpl proxy1 = (PoolImpl) pf.create("clientPool1");
+     try {
+     PoolImpl proxy2 = (PoolImpl) pf.create("clientPool2");
+     try {
 
      Map map1 = proxy1.getThreadIdToSequenceIdMap();
      Map map2 = proxy2.getThreadIdToSequenceIdMap();
 
      assertTrue(!(map1==map2));
 
-      // Close the proxies to remove them from the set of all proxies
-     proxy1.close();
-     proxy2.close();
+     } finally {
+       proxy2.destroy();
+     }
+     } finally {
+       proxy1.destroy();
+     }
     }
    catch (Exception ex) {
      ex.printStackTrace();
@@ -686,10 +696,10 @@ public class ConnectionProxyJUnitTest
   public void testPeriodicAckSendByClient()
  {
    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
      server.setPort(port);
      server.start();
    }
@@ -698,15 +708,15 @@ public class ConnectionProxyJUnitTest
      fail("Failed to create server");
    }
    try {
-     Properties props = new Properties();
-     props.setProperty("endpoints", "ep1=localhost:"+port);
-     props.setProperty("establishCallbackConnection", "true");
-     props.setProperty("redundancyLevel", "1");
-     props.setProperty("readTimeout", "20000");
-     props.setProperty("messageTrackingTimeout", "15000");
-     props.setProperty("clientAckInterval", "5000");
+     PoolFactory pf = PoolManager.createFactory();
+     pf.addServer("localhost", port);
+     pf.setSubscriptionEnabled(true);
+     pf.setSubscriptionRedundancy(1);
+     pf.setReadTimeout(20000);
+     pf.setSubscriptionMessageTrackingTimeout(15000);
+     pf.setSubscriptionAckInterval(5000);
      
-     proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);   
+     proxy = (PoolImpl) pf.create("clientPool");
      
      EventID eid = new EventID(new byte[0],1,1);
      
@@ -757,10 +767,10 @@ public class ConnectionProxyJUnitTest
   public void testNoAckSendByClient()
  {
    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-   BridgeServer server = null;
+   CacheServer server = null;
    try {
    try {
-     server = this.cache.addBridgeServer();
+     server = this.cache.addCacheServer();
      server.setPort(port);
      server.start();
    }
@@ -769,14 +779,15 @@ public class ConnectionProxyJUnitTest
      fail("Failed to create server");
    }
    try {
-     Properties props = new Properties();
-     props.setProperty("endpoints", "ep1=localhost:"+port);     
-     props.setProperty("readTimeout", "20000");
-     props.setProperty("messageTrackingTimeout", "8000");
-     props.setProperty("clientAckInterval", "2000");
-     props.setProperty("establishCallbackConnection", "true");
+     PoolFactory pf = PoolManager.createFactory();
+     pf.addServer("localhost", port);
+     pf.setSubscriptionEnabled(true);
+     pf.setSubscriptionRedundancy(1);
+     pf.setReadTimeout(20000);
+     pf.setSubscriptionMessageTrackingTimeout(8000);
+     pf.setSubscriptionAckInterval(2000);
      
-     proxy = BridgePoolImpl.create(props, true/*useByBridgeWriter*/);     
+     proxy = (PoolImpl) pf.create("clientPool");
      
      EventID eid = new EventID(new byte[0],1,1);
      

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DataSerializerPropogationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DataSerializerPropogationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DataSerializerPropogationDUnitTest.java
index a8b5e5e..85bd4d1 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DataSerializerPropogationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DataSerializerPropogationDUnitTest.java
@@ -27,16 +27,16 @@ import com.gemstone.gemfire.cache.Scope;
 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.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.InternalDataSerializer.SerializerAttributesHolder;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 
 import dunit.DistributedTestCase;
@@ -132,7 +132,7 @@ public class DataSerializerPropogationDUnitTest extends DistributedTestCase {
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setMaxThreads(maxThreads.intValue());
     server1.setNotifyBySubscription(true);
@@ -353,9 +353,9 @@ public class DataSerializerPropogationDUnitTest extends DistributedTestCase {
 
   public static void stopServer() {
     try {
-      assertEquals("Expected exactly one BridgeServer", 1, cache
-          .getBridgeServers().size());
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
+      assertEquals("Expected exactly one CacheServer", 1, cache
+          .getCacheServers().size());
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
           .iterator().next();
       assertNotNull(bs);
       bs.stop();
@@ -368,9 +368,9 @@ public class DataSerializerPropogationDUnitTest extends DistributedTestCase {
   public static void startServer() {
     try {
       Cache c = CacheFactory.getAnyInstance();
-      assertEquals("Expected exactly one BridgeServer", 1, c.getBridgeServers()
+      assertEquals("Expected exactly one CacheServer", 1, c.getCacheServers()
           .size());
-      BridgeServerImpl bs = (BridgeServerImpl)c.getBridgeServers().iterator()
+      CacheServerImpl bs = (CacheServerImpl)c.getCacheServers().iterator()
           .next();
       assertNotNull(bs);
       bs.start();
@@ -726,9 +726,9 @@ public class DataSerializerPropogationDUnitTest extends DistributedTestCase {
     client2.invoke(DataSerializerPropogationDUnitTest.class,
         "createClientCache", new Object[] {
             getServerHostName(server2.getHost()), new Integer(PORT2) });
-    setBridgeObserver1();
+    setClientServerObserver1();
     client2
-        .invoke(DataSerializerPropogationDUnitTest.class, "setBridgeObserver2");
+        .invoke(DataSerializerPropogationDUnitTest.class, "setClientServerObserver2");
 
     registerDSObject13();
 
@@ -873,7 +873,7 @@ public class DataSerializerPropogationDUnitTest extends DistributedTestCase {
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setMaxThreads(maxThreads.intValue());
     server1.setNotifyBySubscription(true);
@@ -897,7 +897,7 @@ public class DataSerializerPropogationDUnitTest extends DistributedTestCase {
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setMaxThreads(maxThreads.intValue());
     server1.setNotifyBySubscription(true);
@@ -905,9 +905,9 @@ public class DataSerializerPropogationDUnitTest extends DistributedTestCase {
     return new Integer(port);
   }
 
-  public static void setBridgeObserver1() {
+  public static void setClientServerObserver1() {
     PoolImpl.IS_INSTANTIATOR_CALLBACK = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       @Override
       public void beforeSendingToServer(EventID eventID) {
         eventId = eventID;
@@ -928,9 +928,9 @@ public class DataSerializerPropogationDUnitTest extends DistributedTestCase {
     eventId = eventID;
   }
 
-  public static void setBridgeObserver2() {
+  public static void setClientServerObserver2() {
     PoolImpl.IS_INSTANTIATOR_CALLBACK = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       @Override
       public void afterReceivingFromServer(EventID eventID) {
         testEventIDResult = eventID.equals(eventId);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DestroyEntryPropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DestroyEntryPropagationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DestroyEntryPropagationDUnitTest.java
index b0b9912..a8ad9f9 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DestroyEntryPropagationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DestroyEntryPropagationDUnitTest.java
@@ -24,7 +24,6 @@ import com.gemstone.gemfire.cache.Operation;
 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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -36,6 +35,7 @@ import com.gemstone.gemfire.cache.client.*;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
 import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.server.CacheServer;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
@@ -287,10 +287,10 @@ public class DestroyEntryPropagationDUnitTest extends DistributedTestCase
   public static void killServer(Integer port)
   {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
-      getLogWriter().fine ("Asif: servers running = "+cache.getBridgeServers().size());
+      Iterator iter = cache.getCacheServers().iterator();
+      getLogWriter().fine ("Asif: servers running = "+cache.getCacheServers().size());
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         getLogWriter().fine("asif : server running on port="+server.getPort()+ " asked to kill serevre onport="+port);
          if(port.intValue() == server.getPort()){
          server.stop();
@@ -305,7 +305,7 @@ public class DestroyEntryPropagationDUnitTest extends DistributedTestCase
   public static void startServer(Integer port)
   {
     try {
-      BridgeServer server1 = cache.addBridgeServer();
+      CacheServer server1 = cache.addCacheServer();
       server1.setPort(port.intValue());
       server1.setNotifyBySubscription(true);
       server1.start();
@@ -455,7 +455,7 @@ public class DestroyEntryPropagationDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
 
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
     server.setPort(port);
     server.setNotifyBySubscription(true);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
index 76f8d44..8d2ef40 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
@@ -28,12 +28,12 @@ import com.gemstone.gemfire.cache.client.PoolFactory;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.Connection;
 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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 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;
@@ -464,7 +464,7 @@ public class DurableClientReconnectDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     Region r = cache.createRegion(REGION_NAME, attrs);
     assertNotNull(r);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
@@ -563,8 +563,8 @@ public class DurableClientReconnectDUnitTest extends DistributedTestCase
         .getClientProxies().size();
   }
   
-  private static BridgeServerImpl getBridgeServer() {
-    BridgeServerImpl bridgeServer = (BridgeServerImpl) cache.getBridgeServers().iterator().next();
+  private static CacheServerImpl getBridgeServer() {
+    CacheServerImpl bridgeServer = (CacheServerImpl) cache.getCacheServers().iterator().next();
     assertNotNull(bridgeServer);
     return bridgeServer;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java
index 8e3e145..f8f036e 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java
@@ -18,7 +18,7 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.cache.client.*;
 import com.gemstone.gemfire.internal.cache.PoolFactoryImpl;
 
@@ -269,8 +269,8 @@ public class DurableClientStatsDUnitTest extends DistributedTestCase {
     try {
       Cache cache = CacheServerTestUtil.getCache();
       com.gemstone.gemfire.LogWriter logger = cache.getLogger();
-      BridgeServerImpl currentServer = (BridgeServerImpl)(new ArrayList(cache
-          .getBridgeServers()).get(0));
+      CacheServerImpl currentServer = (CacheServerImpl)(new ArrayList(cache
+          .getCacheServers()).get(0));
       AcceptorImpl ai = currentServer.getAcceptor();
       CacheClientNotifier notifier = ai.getCacheClientNotifier();
       CacheClientNotifierStats stats = notifier.getStats();
@@ -290,8 +290,8 @@ public class DurableClientStatsDUnitTest extends DistributedTestCase {
     try {
       Cache cache = CacheServerTestUtil.getCache();
       com.gemstone.gemfire.LogWriter logger = cache.getLogger();
-      BridgeServerImpl currentServer = (BridgeServerImpl)(new ArrayList(cache
-          .getBridgeServers()).get(0));
+      CacheServerImpl currentServer = (CacheServerImpl)(new ArrayList(cache
+          .getCacheServers()).get(0));
       AcceptorImpl ai = currentServer.getAcceptor();
       CacheClientNotifier notifier = ai.getCacheClientNotifier();
       CacheClientNotifierStats stats = notifier.getStats();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableRegistrationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableRegistrationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableRegistrationDUnitTest.java
index 0e1972b..14c7ac4 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableRegistrationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableRegistrationDUnitTest.java
@@ -22,7 +22,7 @@ import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 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.FilterProfile;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
@@ -835,9 +835,9 @@ public class DurableRegistrationDUnitTest extends DistributedTestCase {
     return proxy;
   }
 
-  private static BridgeServerImpl getBridgeServer() {
-    BridgeServerImpl bridgeServer = (BridgeServerImpl)CacheServerTestUtil
-        .getCache().getBridgeServers().iterator().next();
+  private static CacheServerImpl getBridgeServer() {
+    CacheServerImpl bridgeServer = (CacheServerImpl)CacheServerTestUtil
+        .getCache().getCacheServers().iterator().next();
     assertNotNull(bridgeServer);
     return bridgeServer;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableResponseMatrixDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableResponseMatrixDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableResponseMatrixDUnitTest.java
index 0ccee9e..79f7281 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableResponseMatrixDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableResponseMatrixDUnitTest.java
@@ -10,11 +10,11 @@ package com.gemstone.gemfire.internal.cache.tier.sockets;
 import java.util.Properties;
 
 import com.gemstone.gemfire.cache.*;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.cache.client.*;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
@@ -147,7 +147,7 @@ public class DurableResponseMatrixDUnitTest extends DistributedTestCase
   public void testRegisterInterest_Destroy_Concurrent() throws Exception
   {  
 	PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = true;
-	BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+	ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
 		public void beforeInterestRegistration()
 	    {	          
 	      	Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME);	  
@@ -474,7 +474,7 @@ public class DurableResponseMatrixDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     Region r = cache.createRegion(REGION_NAME, attrs);
     assertNotNull(r);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server1.setPort(port);
     server1.setNotifyBySubscription(true);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/EventIDVerificationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/EventIDVerificationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/EventIDVerificationDUnitTest.java
index 7e162f6..c52e7a5 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/EventIDVerificationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/EventIDVerificationDUnitTest.java
@@ -21,10 +21,10 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionEvent;
 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.cache.util.CacheWriterAdapter;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -174,10 +174,10 @@ public class EventIDVerificationDUnitTest extends DistributedTestCase
     factory.setScope(Scope.DISTRIBUTED_ACK);
     factory.setMirrorType(MirrorType.NONE);
     
-    BridgeTestCase.configureConnectionPool(factory, host, new int[] {PORT1,PORT2}, true, -1, 2, null, -1, -1, false, -2);
+    ClientServerTestCase.configureConnectionPool(factory, host, new int[] {PORT1,PORT2}, true, -1, 2, null, -1, -1, false, -2);
 
     
-    CacheWriter writer = new CacheWriterAdapter() { // couldn't port this test to pool because it subclasses BridgeWriter
+    CacheWriter writer = new CacheWriterAdapter() {
       public void beforeCreate(EntryEvent event)
       {
         vm0.invoke(EventIDVerificationDUnitTest.class, "setEventIDData",
@@ -360,7 +360,7 @@ public class EventIDVerificationDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
     server1.start();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateEvictionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateEvictionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateEvictionDUnitTest.java
index 1f6c123..4a2c003 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateEvictionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ForceInvalidateEvictionDUnitTest.java
@@ -111,7 +111,7 @@ public class ForceInvalidateEvictionDUnitTest extends CacheTestCase {
     
     createPR(vm0);
     createPR(vm1);
-    int port = addBridgeServer(vm1);
+    int port = addCacheServer(vm1);
     createClient(vm3, port);
     
     
@@ -132,7 +132,7 @@ public class ForceInvalidateEvictionDUnitTest extends CacheTestCase {
     createPR(vm0);
     createPR(vm1);
     createAccessor(vm2, false);
-    int port = addBridgeServer(vm2);
+    int port = addCacheServer(vm2);
     createClient(vm3, port);
     
     doPropagationTest(vm0, vm3, true, true);
@@ -151,7 +151,7 @@ public class ForceInvalidateEvictionDUnitTest extends CacheTestCase {
     createAccessor(vm2, false);
     
     //test an invalidate from the accessor through one of the data stores
-    int port1 = addBridgeServer(vm0);
+    int port1 = addCacheServer(vm0);
     createClient(vm3, port1);
     doPropagationTest(vm2, vm3, true, true);
     vm3.invoke(new SerializableRunnable("close cache") {
@@ -163,7 +163,7 @@ public class ForceInvalidateEvictionDUnitTest extends CacheTestCase {
     });
     
     //test an invalidate from the accessor through the other data store
-    int port2 = addBridgeServer(vm1);
+    int port2 = addCacheServer(vm1);
     createClient(vm3, port2);
     doPropagationTest(vm2, vm3, true, true);
   }
@@ -347,7 +347,7 @@ public class ForceInvalidateEvictionDUnitTest extends CacheTestCase {
     
   }
 
-  private int addBridgeServer(VM vm) {
+  private int addCacheServer(VM vm) {
     final int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     vm.invoke(new SerializableRunnable("add bridge server") {
       public void run() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
index 9aafaed..c8d5fcc 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestBaseTest.java
@@ -20,14 +20,14 @@ import com.gemstone.gemfire.cache.client.internal.Connection;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.RegisterInterestTracker;
 import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.InterestType;
 
@@ -285,9 +285,9 @@ public class HAInterestBaseTest extends DistributedTestCase {
     r1.put(k2, server_k2);
   }
 
-  public static void setBridgeObserverForBeforeInterestRecoveryFailure() {
+  public static void setClientServerObserverForBeforeInterestRecoveryFailure() {
     PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       public void beforeInterestRecovery() {
         synchronized (HAInterestBaseTest.class) {
           Thread t = new Thread() {
@@ -310,9 +310,9 @@ public class HAInterestBaseTest extends DistributedTestCase {
     });
   }
 
-  public static void setBridgeObserverForBeforeInterestRecovery() {
+  public static void setClientServerObserverForBeforeInterestRecovery() {
     PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       public void beforeInterestRecovery() {
         synchronized (HAInterestBaseTest.class) {
           Thread t = new Thread() {
@@ -341,9 +341,9 @@ public class HAInterestBaseTest extends DistributedTestCase {
     }
   }
 
-  public static void setBridgeObserverForBeforeRegistration(final VM vm) {
+  public static void setClientServerObserverForBeforeRegistration(final VM vm) {
     PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       public void beforeInterestRegistration() {
         synchronized (HAInterestBaseTest.class) {
           vm.invoke(HAInterestBaseTest.class, "startServer");
@@ -364,9 +364,9 @@ public class HAInterestBaseTest extends DistributedTestCase {
     }
   }
 
-  public static void setBridgeObserverForAfterRegistration(final VM vm) {
+  public static void setClientServerObserverForAfterRegistration(final VM vm) {
     PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       public void afterInterestRegistration() {
         synchronized (HAInterestBaseTest.class) {
           vm.invoke(HAInterestBaseTest.class, "startServer");
@@ -389,7 +389,7 @@ public class HAInterestBaseTest extends DistributedTestCase {
     }
   }
 
-  public static void unSetBridgeObserverForRegistrationCallback() {
+  public static void unSetClientServerObserverForRegistrationCallback() {
     synchronized (HAInterestBaseTest.class) {
       PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG = false;
       PoolImpl.AFTER_REGISTER_CALLBACK_FLAG = false;
@@ -399,21 +399,21 @@ public class HAInterestBaseTest extends DistributedTestCase {
   }
 
   public static void verifyDispatcherIsAlive() {
-    assertEquals("More than one BridgeServer", 1, cache.getBridgeServers().size());
+    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
     
     WaitCriterion wc = new WaitCriterion() {
       @Override
       public boolean done() {
-        return cache.getBridgeServers().size() == 1;
+        return cache.getCacheServers().size() == 1;
       }
       @Override
       public String description() {
-        return "waiting for cache.getBridgeServers().size() == 1";
+        return "waiting for cache.getCacheServers().size() == 1";
       }
     };
     DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
 
-    BridgeServerImpl bs = (BridgeServerImpl) cache.getBridgeServers().iterator().next();
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
     assertNotNull(bs);
     assertNotNull(bs.getAcceptor());
     assertNotNull(bs.getAcceptor().getCacheClientNotifier());
@@ -458,16 +458,16 @@ public class HAInterestBaseTest extends DistributedTestCase {
     WaitCriterion wc = new WaitCriterion() {
       @Override
       public boolean done() {
-        return cache.getBridgeServers().size() == 1;
+        return cache.getCacheServers().size() == 1;
       }
       @Override
       public String description() {
-        return "cache.getBridgeServers().size() == 1";
+        return "cache.getCacheServers().size() == 1";
       }
     };
     DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
 
-    BridgeServerImpl bs = (BridgeServerImpl) cache.getBridgeServers().iterator().next();
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
     assertNotNull(bs);
     assertNotNull(bs.getAcceptor());
     assertNotNull(bs.getAcceptor().getCacheClientNotifier());
@@ -551,15 +551,15 @@ public class HAInterestBaseTest extends DistributedTestCase {
 
   public static void startServer() throws IOException {
     Cache c = CacheFactory.getAnyInstance();
-    assertEquals("More than one BridgeServer", 1, c.getBridgeServers().size());
-    BridgeServerImpl bs = (BridgeServerImpl) c.getBridgeServers().iterator().next();
+    assertEquals("More than one BridgeServer", 1, c.getCacheServers().size());
+    CacheServerImpl bs = (CacheServerImpl) c.getCacheServers().iterator().next();
     assertNotNull(bs);
     bs.start();
   }
 
   public static void stopServer() {
-    assertEquals("More than one BridgeServer", 1, cache.getBridgeServers().size());
-    BridgeServerImpl bs = (BridgeServerImpl) cache.getBridgeServers().iterator().next();
+    assertEquals("More than one BridgeServer", 1, cache.getCacheServers().size());
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
     assertNotNull(bs);
     bs.stop();
   }
@@ -749,16 +749,16 @@ public class HAInterestBaseTest extends DistributedTestCase {
     WaitCriterion wc = new WaitCriterion() {
       @Override
       public boolean done() {
-        return cache.getBridgeServers().size() == 1;
+        return cache.getCacheServers().size() == 1;
       }
       @Override
       public String description() {
-        return "waiting for cache.getBridgeServers().size() == 1";
+        return "waiting for cache.getCacheServers().size() == 1";
       }
     };
     DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
 
-    BridgeServerImpl bs = (BridgeServerImpl) cache.getBridgeServers().iterator().next();
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
     assertNotNull(bs);
     assertNotNull(bs.getAcceptor());
     assertNotNull(bs.getAcceptor().getCacheClientNotifier());
@@ -809,16 +809,16 @@ public class HAInterestBaseTest extends DistributedTestCase {
     WaitCriterion wc = new WaitCriterion() {
       @Override
       public boolean done() {
-        return cache.getBridgeServers().size() == 1;
+        return cache.getCacheServers().size() == 1;
       }
       @Override
       public String description() {
-        return "waiting for cache.getBridgeServers().size() == 1";
+        return "waiting for cache.getCacheServers().size() == 1";
       }
     };
     DistributedTestCase.waitForCriterion(wc, TIMEOUT_MILLIS, INTERVAL_MILLIS, true);
 
-    BridgeServerImpl bs = (BridgeServerImpl) cache.getBridgeServers().iterator().next();
+    CacheServerImpl bs = (CacheServerImpl) cache.getCacheServers().iterator().next();
     assertNotNull(bs);
     assertNotNull(bs.getAcceptor());
     assertNotNull(bs.getAcceptor().getCacheClientNotifier());
@@ -976,7 +976,7 @@ public class HAInterestBaseTest extends DistributedTestCase {
     factory.setConcurrencyChecksEnabled(true);
     cache.createRegion(REGION_NAME, factory.create());
 
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
     server.setMaximumTimeBetweenPings(180000);
@@ -994,7 +994,7 @@ public class HAInterestBaseTest extends DistributedTestCase {
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
 
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
     // ensures updates to be sent instead of invalidations

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
index 2ea7f7c..995bc31 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAInterestPart2DUnitTest.java
@@ -82,12 +82,12 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
     VM backup = getBackupVM();
     backup.invoke(HAInterestBaseTest.class, "stopServer");
     verifyDeadAndLiveServers(1, 2);
-    setBridgeObserverForBeforeRegistration(backup);
+    setClientServerObserverForBeforeRegistration(backup);
     try {
       registerK1AndK2();
       waitForBeforeRegistrationCallback();
     } finally {
-      unSetBridgeObserverForRegistrationCallback();
+      unSetClientServerObserverForRegistrationCallback();
     }
     server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
     server2.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
@@ -111,12 +111,12 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
     backup.invoke(HAInterestBaseTest.class, "stopServer");
     verifyDeadAndLiveServers(1, 2);
 
-    setBridgeObserverForAfterRegistration(backup);
+    setClientServerObserverForAfterRegistration(backup);
     try {
       registerK1AndK2();
       waitForAfterRegistrationCallback();
     } finally {
-      unSetBridgeObserverForRegistrationCallback();
+      unSetClientServerObserverForRegistrationCallback();
     }
 
     server1.invoke(HAInterestBaseTest.class, "verifyInterestRegistration");
@@ -248,7 +248,7 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
     // put on stopped server
     server1.invoke(HAInterestBaseTest.class, "putK1andK2");
     // spawn a thread to put on server , which will acquire a lock on entry
-    setBridgeObserverForBeforeInterestRecovery();
+    setClientServerObserverForBeforeInterestRecovery();
     server1.invoke(HAInterestBaseTest.class, "startServer");
     verifyDeadAndLiveServers(0, 1);
     waitForBeforeInterestRecoveryCallBack();
@@ -304,7 +304,7 @@ public class HAInterestPart2DUnitTest extends HAInterestBaseTest {
     verifyDeadAndLiveServers(2, 0);
 
     primary.invoke(HAInterestBaseTest.class, "putK1andK2");
-    setBridgeObserverForBeforeInterestRecoveryFailure();
+    setClientServerObserverForBeforeInterestRecoveryFailure();
     primary.invoke(HAInterestBaseTest.class, "startServer");
     waitForBeforeInterestRecoveryCallBack();
     if (exceptionOccured) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
index e83b300..4242fe0 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/HAStartupAndFailoverDUnitTest.java
@@ -22,14 +22,14 @@ 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.client.internal.Connection;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
@@ -98,23 +98,23 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       server2.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsNotAlive");
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsNotAlive");
 
-      setBridgeObserver();
+      setClientServerObserver();
 
       server1.invoke(HAStartupAndFailoverDUnitTest.class, "stopServer");
 
       waitForPrimaryIdentification();
       //primary
       server2.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsAlive");
-      unSetBridgeObserver();
+      unSetClientServerObserver();
       //secondary
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsNotAlive");
 
-      setBridgeObserver();
+      setClientServerObserver();
       server2.invoke(HAStartupAndFailoverDUnitTest.class, "stopServer");
       //primary
       waitForPrimaryIdentification();
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsAlive");
-      unSetBridgeObserver();
+      unSetClientServerObserver();
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "stopServer");
       // All servers are dead at this point , no primary in the system.
       verifyDeadAndLiveServers(3,0);
@@ -146,7 +146,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       //secondary
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsNotAlive");
 
-      setBridgeObserver();
+      setClientServerObserver();
 
       //stop new primary
       server2.invoke(HAStartupAndFailoverDUnitTest.class, "stopServer");
@@ -156,7 +156,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       //newly selectd primary
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsAlive");
 
-      unSetBridgeObserver();
+      unSetClientServerObserver();
     }
 
     /**
@@ -174,7 +174,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       // secondaries
       server2.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsNotAlive");
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsNotAlive");
-      setBridgeObserver();
+      setClientServerObserver();
 
       server1.invoke(HAStartupAndFailoverDUnitTest.class, "stopServer");
       //stop ProbablePrimary
@@ -185,7 +185,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       waitForPrimaryIdentification();
       //new primary
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsAlive");
-      unSetBridgeObserver();
+      unSetClientServerObserver();
 
     }
 
@@ -212,10 +212,10 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       server1.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsAlive");
       server2.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsNotAlive");
       server3.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsNotAlive");
-      setBridgeObserver();
+      setClientServerObserver();
       server1.invoke(HAStartupAndFailoverDUnitTest.class, "stopServer");
       waitForPrimaryIdentification();
-      unSetBridgeObserver();
+      unSetClientServerObserver();
       verifyDeadAndLiveServers(1,2);
       server2.invoke(HAStartupAndFailoverDUnitTest.class, "verifyDispatcherIsAlive");
      }
@@ -288,31 +288,6 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       verifyDeadAndLiveServers(1,2);
     }
 
-
-  // darrel: this following is an invalid test.
-  // A "primary" is only identified when you have a callback connection
-//     /**
-//      * Tests failover initialization by cache operation Threads on Primary
-//      */
-//     public void testInitiateFailoverByCacheOperationThreads_Primary() throws Exception
-//     {
-//       // create a client with large retry interval for server monitors and no client updater thread
-//       // so that only cache operation can detect a server failure and should initiate failover
-//       createClientCacheWithLargeRetryIntervalAndWithoutCallbackConnection(this.getName());
-//       setBridgeObserver();
-//       server1.invoke(HAStartupAndFailoverDUnitTest.class, "stopServer");
-//       releaseConnection();//Added by Jason
-//       put();
-//       waitForPrimaryIdentification();
-//       unSetBridgeObserver();
-//       verifyDeadAndLiveServers(1,2);
-//     }
-//     public static void releaseConnection() {
-//       Region r1 = cache.getRegion("/" + REGION_NAME);
-//       BridgeWriter bw = (BridgeWriter)r1.getAttributes().getCacheWriter();
-//       bw.release();
-//     }
-
     public static void put()
     {
       try {
@@ -357,9 +332,9 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
 //     start = System.currentTimeMillis();
   }
  
-    public static void setBridgeObserver() {
+    public static void setClientServerObserver() {
         PoolImpl.AFTER_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = true;
-        BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+        ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
             public void afterPrimaryIdentificationFromBackup(ServerLocation primaryEndpoint) {
                 synchronized (HAStartupAndFailoverDUnitTest.class) {
                   HAStartupAndFailoverDUnitTest.identifiedPrimary = true;
@@ -369,12 +344,12 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
         });
     }
 
-    public static void unSetBridgeObserver()
+    public static void unSetClientServerObserver()
   {
       synchronized (HAStartupAndFailoverDUnitTest.class) {
           PoolImpl.AFTER_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = false;
           HAStartupAndFailoverDUnitTest.identifiedPrimary = false;
-          BridgeObserverHolder.setInstance(new BridgeObserverAdapter());
+          ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter());
       }
 
   }
@@ -383,8 +358,8 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
     {
     try {
       assertEquals("Expected exactly one BridgeServer", 1, cache
-          .getBridgeServers().size());
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
+          .getCacheServers().size());
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
           .iterator().next();
       assertNotNull(bs);
       bs.stop();
@@ -430,8 +405,8 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
     try {
       Cache c = CacheFactory.getAnyInstance();
       assertEquals("Expected exactly one BridgeServer", 1,
-          c.getBridgeServers().size());
-      BridgeServerImpl bs = (BridgeServerImpl) c.getBridgeServers()
+          c.getCacheServers().size());
+      CacheServerImpl bs = (CacheServerImpl) c.getCacheServers()
           .iterator().next();
       assertNotNull(bs);
       bs.start();
@@ -473,7 +448,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       WaitCriterion wc = new WaitCriterion() {
         String excuse;
         public boolean done() {
-          return cache.getBridgeServers().size() == 1;
+          return cache.getCacheServers().size() == 1;
         }
         public String description() {
           return excuse;
@@ -481,7 +456,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       };
       DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
       
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
           .iterator().next();
       assertNotNull(bs);
       assertNotNull(bs.getAcceptor());
@@ -529,11 +504,11 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
     try {
       Cache c = CacheFactory.getAnyInstance();
       // assertEquals("More than one BridgeServer", 1,
-      // c.getBridgeServers().size());
+      // c.getCacheServers().size());
       WaitCriterion wc = new WaitCriterion() {
         String excuse;
         public boolean done() {
-          return cache.getBridgeServers().size() == 1;
+          return cache.getCacheServers().size() == 1;
         }
         public String description() {
           return excuse;
@@ -541,7 +516,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
       };
       DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
       
-      BridgeServerImpl bs = (BridgeServerImpl)c.getBridgeServers().iterator()
+      CacheServerImpl bs = (CacheServerImpl)c.getCacheServers().iterator()
           .next();
       assertNotNull(bs);
       assertNotNull(bs.getAcceptor());
@@ -714,7 +689,7 @@ public class HAStartupAndFailoverDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
 
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
     server1.setPort(port);
     // ensures updates to be sent instead of invalidations

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListDUnitTest.java
index 493840b..d1ba9d6 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListDUnitTest.java
@@ -30,7 +30,7 @@ import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.cache.client.*;
 import com.gemstone.gemfire.cache.NoSubscriptionServersAvailableException;
 
@@ -566,7 +566,7 @@ public class InterestListDUnitTest extends DistributedTestCase
 
   /** wait for queues to drain in the server */
   public static void flushQueues() throws Exception {
-    BridgeServerImpl impl = (BridgeServerImpl)server;
+    CacheServerImpl impl = (CacheServerImpl)server;
     for (CacheClientProxy proxy: (Set<CacheClientProxy>)impl.getAllClientSessions()) {
       final CacheClientProxy fproxy = proxy;
       WaitCriterion ev = new WaitCriterion() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListEndpointDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListEndpointDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListEndpointDUnitTest.java
index 21d40bc..ca34569 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListEndpointDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListEndpointDUnitTest.java
@@ -29,12 +29,12 @@ import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.Connection;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.QueueConnectionImpl;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 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;
@@ -263,8 +263,8 @@ public class InterestListEndpointDUnitTest extends DistributedTestCase
   {
   try {
         Cache c = CacheFactory.getAnyInstance();
-        assertEquals("More than one BridgeServer", 1, c.getBridgeServers().size());
-        BridgeServerImpl bs = (BridgeServerImpl) c.getBridgeServers().iterator().next();
+        assertEquals("More than one BridgeServer", 1, c.getCacheServers().size());
+        CacheServerImpl bs = (CacheServerImpl) c.getCacheServers().iterator().next();
         assertNotNull(bs);
         assertNotNull(bs.getAcceptor());
         assertNotNull(bs.getAcceptor().getCacheClientNotifier());
@@ -273,9 +273,9 @@ public class InterestListEndpointDUnitTest extends DistributedTestCase
        CacheClientProxy proxy = (CacheClientProxy)iter_prox.next();
        //if (proxy._interestList._keysOfInterest.get("/"+REGION_NAME) != null) {
        if(proxy.isPrimary()){
-          Iterator iter = cache.getBridgeServers().iterator();
+          Iterator iter = cache.getCacheServers().iterator();
           if (iter.hasNext()) {
-            BridgeServer server = (BridgeServer)iter.next();
+            CacheServer server = (CacheServer)iter.next();
                   cache.getLogger().fine("stopping server " + server);
             server.stop();
           }
@@ -356,7 +356,7 @@ public class InterestListEndpointDUnitTest extends DistributedTestCase
     new InterestListEndpointDUnitTest("temp").createCache(new Properties());
     RegionAttributes attrs = impl.createServerCacheAttributes();
     cache.createRegion(REGION_NAME, attrs);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
     server1.setPort(port);
     server1.setMaxThreads(maxThreads.intValue());
@@ -390,8 +390,8 @@ public class InterestListEndpointDUnitTest extends DistributedTestCase
   {
     try {
       Cache c = CacheFactory.getAnyInstance();
-      assertEquals("More than one BridgeServer", 1, c.getBridgeServers().size());
-      BridgeServerImpl bs = (BridgeServerImpl) c.getBridgeServers().iterator().next();
+      assertEquals("More than one CacheServer", 1, c.getCacheServers().size());
+      CacheServerImpl bs = (CacheServerImpl) c.getCacheServers().iterator().next();
       assertNotNull(bs);
       assertNotNull(bs.getAcceptor());
       assertNotNull(bs.getAcceptor().getCacheClientNotifier());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java
index 28923c1..985abb2 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java
@@ -12,7 +12,7 @@ package com.gemstone.gemfire.internal.cache.tier.sockets;
 import java.util.Iterator;
 
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.client.*;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.cache.PoolFactoryImpl;
@@ -202,9 +202,9 @@ public class InterestListFailoverDUnitTest extends DistributedTestCase
   public static void stopServer()
   {
     try {
-      Iterator iter = CacheServerTestUtil.getCache().getBridgeServers().iterator();
+      Iterator iter = CacheServerTestUtil.getCache().getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
           server.stop();
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListRecoveryDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListRecoveryDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListRecoveryDUnitTest.java
index 60e7a63..ebaba34 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListRecoveryDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListRecoveryDUnitTest.java
@@ -24,11 +24,11 @@ import com.gemstone.gemfire.cache.InterestResultPolicy;
 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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 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.LocalRegion;
 import com.gemstone.gemfire.cache.client.*;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
@@ -226,7 +226,7 @@ public class InterestListRecoveryDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
     server1.start();
@@ -452,8 +452,8 @@ public class InterestListRecoveryDUnitTest extends DistributedTestCase
 
  public static Set getCacheClientProxies() {
    Cache c = CacheFactory.getAnyInstance();
-   assertEquals("More than one BridgeServer", 1, c.getBridgeServers().size());
-   BridgeServerImpl bs = (BridgeServerImpl)c.getBridgeServers().iterator()
+   assertEquals("More than one CacheServer", 1, c.getCacheServers().size());
+   CacheServerImpl bs = (CacheServerImpl)c.getCacheServers().iterator()
    .next();
    assertNotNull(bs);
    assertNotNull(bs.getAcceptor());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestResultPolicyDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestResultPolicyDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestResultPolicyDUnitTest.java
index 42c4bfa..dbdc1cc 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestResultPolicyDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestResultPolicyDUnitTest.java
@@ -20,7 +20,7 @@ import com.gemstone.gemfire.cache.InterestResultPolicy;
 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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -244,7 +244,7 @@ public class InterestResultPolicyDUnitTest extends DistributedTestCase
     factory.setScope(Scope.DISTRIBUTED_ACK);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
     server.setNotifyBySubscription(true);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.java
index c448511..4cb31e7 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.java
@@ -21,7 +21,7 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.Pool;
-import com.gemstone.gemfire.cache.util.BridgeWriter;
+import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.cache.tier.ConnectionProxy;
@@ -46,9 +46,6 @@ public class RedundancyLevelJUnitTest
   /** The distributed system */
   Cache cache;
 
-  /** The proxy instance */
-  ConnectionProxy proxy = null;
-
   /**
    * Close the cache and proxy instances for a test and disconnect from the
    * distributed system.
@@ -71,9 +68,6 @@ public class RedundancyLevelJUnitTest
       
       system.disconnect();
     }
-    if (proxy != null) {
-      proxy.close();
-    }
   }
 
   /**
@@ -114,16 +108,14 @@ public class RedundancyLevelJUnitTest
         assertNotNull("cache was null", cache);
         Region region = cache.getRegion("/root/exampleRegion");
         assertNotNull(region);
-        BridgeWriter writer = (BridgeWriter)region.getAttributes()
-            .getCacheWriter();
-        Pool pool = (Pool)writer.getConnectionProxy();
+        Pool pool = PoolManager.find("clientPool");
         assertEquals(
             "Redundancy level not matching the one specified in cache-xml", 6,
             pool.getSubscriptionRedundancy());
       } finally {
         final String removeExpected =
           "<ExpectedException action=remove>" + expected + "</ExpectedException>";
-        cache.getLogger().info(removeExpected);
+        system.getLogWriter().info(removeExpected);
       }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java
index c1f4aab..05f5449 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart1DUnitTest.java
@@ -81,7 +81,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
    */
   public void testRedundancyNotSpecifiedPrimaryServerFails()
   {
-    /*BridgeObserver oldBo = BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    /*ClientServerObserver oldBo = ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       public void beforeFailoverByCacheClientUpdater(Endpoint epFailed)
       {
         try{
@@ -123,7 +123,7 @@ public class RedundancyLevelPart1DUnitTest extends RedundancyLevelTestBase
           "test failed due to exception in test RedundancyNotSpecifiedPrimaryServerFails ",
           ex);
     }/*finally {
-      BridgeObserverHolder.setInstance(oldBo);
+      ClientServerObserverHolder.setInstance(oldBo);
     }*/
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart3DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart3DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart3DUnitTest.java
index 7869289..3d11b5b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart3DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelPart3DUnitTest.java
@@ -7,8 +7,8 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets;
 
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 
 import dunit.DistributedTestCase;
@@ -59,7 +59,7 @@ public class RedundancyLevelPart3DUnitTest extends RedundancyLevelTestBase
       PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
       registerInterestCalled = false;
       makePrimaryCalled = false;
-      BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+      ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
         public void beforeInterestRegistration()
         {
           registerInterestCalled = true;


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterDataSerializersOp.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterDataSerializersOp.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterDataSerializersOp.java
index c3ab84b..d06107d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterDataSerializersOp.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterDataSerializersOp.java
@@ -12,8 +12,8 @@ import java.io.IOException;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.SerializationException;
 import com.gemstone.gemfire.internal.InternalDataSerializer.SerializerAttributesHolder;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
@@ -61,7 +61,7 @@ public class RegisterDataSerializersOp {
       getMessage().addBytesPart(eventId.calcBytes());
       // // CALLBACK FOR TESTING PURPOSE ONLY ////
       if (PoolImpl.IS_INSTANTIATOR_CALLBACK) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.beforeSendingToServer(eventId);
       }
    }
@@ -85,7 +85,7 @@ public class RegisterDataSerializersOp {
       getMessage().addBytesPart(eventId.calcBytes());
       // // CALLBACK FOR TESTING PURPOSE ONLY ////
       if (PoolImpl.IS_INSTANTIATOR_CALLBACK) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.beforeSendingToServer(eventId);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterInstantiatorsOp.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterInstantiatorsOp.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterInstantiatorsOp.java
index f3c9f03..8e7515d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterInstantiatorsOp.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/RegisterInstantiatorsOp.java
@@ -12,8 +12,8 @@ import java.io.IOException;
 import com.gemstone.gemfire.Instantiator;
 import com.gemstone.gemfire.SerializationException;
 import com.gemstone.gemfire.internal.InternalInstantiator.InstantiatorAttributesHolder;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
@@ -87,7 +87,7 @@ public class RegisterInstantiatorsOp {
       getMessage().addBytesPart(eventId.calcBytes());
 //     // // CALLBACK FOR TESTING PURPOSE ONLY ////
       if (PoolImpl.IS_INSTANTIATOR_CALLBACK) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.beforeSendingToServer(eventId);
       }
     }
@@ -128,7 +128,7 @@ public class RegisterInstantiatorsOp {
       getMessage().addBytesPart(eventId.calcBytes());
       // // // CALLBACK FOR TESTING PURPOSE ONLY ////
       if (PoolImpl.IS_INSTANTIATOR_CALLBACK) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.beforeSendingToServer(eventId);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerRegionProxy.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerRegionProxy.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerRegionProxy.java
index 7670ef9..b10a38b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerRegionProxy.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ServerRegionProxy.java
@@ -27,13 +27,10 @@ import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.ContainsKeyOp.MODE;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
-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.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.cache.AbstractRegion;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
@@ -87,37 +84,8 @@ public class ServerRegionProxy extends ServerProxy implements ServerRegionDataAc
   private static InternalPool calcPool(Region r) {
     String poolName = r.getAttributes().getPoolName();
     if (poolName == null || "".equals(poolName)) {
-      final CacheLoader cl = r.getAttributes().getCacheLoader();
-      final CacheWriter cw = r.getAttributes().getCacheWriter();
-      if (AbstractRegion.isBridgeLoader(cl) || AbstractRegion.isBridgeWriter(cw)) {
-        Object loaderPool = null;
-        Object writerPool = null;
-        if (AbstractRegion.isBridgeLoader(cl)) {
-          if (cl instanceof BridgeLoader) {
-            loaderPool = ((BridgeLoader)cl).getConnectionProxy();
-          } else {
-            loaderPool = ((BridgeClient)cl).getConnectionProxy();
-          }
-        }
-        if (AbstractRegion.isBridgeWriter(cw)) {
-          writerPool = ((BridgeWriter)cw).getConnectionProxy();
-        }
-        if (loaderPool != writerPool && loaderPool != null && writerPool != null) {
-          throw new IllegalStateException("The region " + r.getFullPath()
-                                          + " has a BridgeLoader and a BridgeWriter/BridgeClient "
-                                          + " that are configured with different connection pools. "
-                                          + " This is not allowed. Instead create a single BridgeClient and install it as both the loader and the writer."
-                                          + " loaderPool="+loaderPool + " writerPool=" + writerPool);
-        }
-        InternalPool result = (InternalPool)loaderPool;
-        if (result == null) {
-          result = (InternalPool)writerPool;
-        }
-        return result;
-      } else {
-        throw new IllegalStateException("The region " + r.getFullPath()
-                                        + " did not have a client pool configured.");
-      }
+      throw new IllegalStateException("The region " + r.getFullPath()
+          + " did not have a client pool configured.");
     } else {
       InternalPool pool = (InternalPool)PoolManager.find(poolName);
       if (pool == null) {
@@ -400,7 +368,7 @@ public class ServerRegionProxy extends ServerProxy implements ServerRegionDataAc
             interestType, policy, isDurable, receiveUpdatesAsInvalidates, regionDataPolicy);
         //////// TEST PURPOSE ONLY ///////////
         if (PoolImpl.AFTER_REGISTER_CALLBACK_FLAG) {
-          BridgeObserver bo = BridgeObserverHolder.getInstance();
+          ClientServerObserver bo = ClientServerObserverHolder.getInstance();
           bo.afterInterestRegistration();
         }
         /////////////////////////////////////////
@@ -594,7 +562,7 @@ public class ServerRegionProxy extends ServerProxy implements ServerRegionDataAc
       finished = true;
       //////// TEST PURPOSE ONLY ///////////
       if (PoolImpl.AFTER_REGISTER_CALLBACK_FLAG) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.afterInterestRegistration();
       }
       /////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java
index afb8b05..af73acc 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/execute/internal/FunctionServiceManager.java
@@ -422,8 +422,7 @@ public final class FunctionServiceManager {
    */
   private final boolean isClientRegion(Region region) {
     LocalRegion localRegion = (LocalRegion) region;
-    return (localRegion.hasServerProxy() || AbstractRegion.isBridgeLoader(localRegion.getCacheLoader()) || AbstractRegion
-        .isBridgeWriter(localRegion.getCacheWriter()));
+    return localRegion.hasServerProxy();
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerLoadProbe.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerLoadProbe.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerLoadProbe.java
index de67c9b..f6e802f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerLoadProbe.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerLoadProbe.java
@@ -11,8 +11,8 @@ import com.gemstone.gemfire.cache.CacheCallback;
 
 
 /**
- * A load probe is installed in a bridge server to measure the load on the
- * bridge server for balancing load between multiple bridge servers.
+ * A load probe is installed in a cache server to measure the load on the
+ * cache server for balancing load between multiple cache servers.
  * 
  * <p>
  * The getLoad method will be called once per poll interval see
@@ -37,7 +37,7 @@ import com.gemstone.gemfire.cache.CacheCallback;
  * Because cache servers can be stopped, reconfigured, and restarted, the open
  * and close methods on this callback can be called several times. If the same
  * callback object is installed on multiple cache servers, open and close will
- * be called once for each bridge server.
+ * be called once for each cache server.
  * </p>
  * 
  * @author dsmith
@@ -53,13 +53,13 @@ public interface ServerLoadProbe extends CacheCallback {
    */
   ServerLoad getLoad(ServerMetrics metrics);
   
-  /** Signals that a bridge server
+  /** Signals that a cache server
    * using this load probe has been started.
    */
   void open();
   
   /**
-   * Signals that a bridge server
+   * Signals that a cache server
    * using this load probe has been closed.
    */
   void close();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerMetrics.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerMetrics.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerMetrics.java
index eb977aa..d4559c8 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerMetrics.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/ServerMetrics.java
@@ -9,7 +9,7 @@ package com.gemstone.gemfire.cache.server;
 
 
 /**
- * Metrics about the resource usage for a bridge server.
+ * Metrics about the resource usage for a cache server.
  * These metrics are provided to the {@link ServerLoadProbe} for
  * use in calculating the load on the server.
  * @author dsmith
@@ -19,23 +19,23 @@ package com.gemstone.gemfire.cache.server;
 public interface ServerMetrics {
   /**
    * Get the number of open connections
-   * for this bridge server.
+   * for this cache server.
    */
   int getConnectionCount();
   
   /** Get the number of clients connected to this
-   * bridge server.
+   * cache server.
    */ 
   int getClientCount();
   
   /**
    * Get the number of client subscription connections hosted on this
-   * bridge server.
+   * cache server.
    */
   int getSubscriptionConnectionCount();
   
   /**
-   * Get the max connections for this bridge server.
+   * Get the max connections for this cache server.
    */
   int getMaxConnections();
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/internal/LoadMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/internal/LoadMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/internal/LoadMonitor.java
index 6210f60..129ba2c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/internal/LoadMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/server/internal/LoadMonitor.java
@@ -14,12 +14,12 @@ import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.cache.client.internal.BridgeServerLoadMessage;
+import com.gemstone.gemfire.cache.client.internal.CacheServerLoadMessage;
 import com.gemstone.gemfire.cache.server.ServerLoad;
 import com.gemstone.gemfire.cache.server.ServerLoadProbe;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
-import com.gemstone.gemfire.internal.cache.BridgeServerAdvisor;
+import com.gemstone.gemfire.internal.cache.CacheServerAdvisor;
 import com.gemstone.gemfire.internal.cache.tier.Acceptor;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
@@ -41,14 +41,14 @@ public class LoadMonitor implements ConnectionListener {
   
   private final ServerLoadProbe probe;
   private final ServerMetricsImpl metrics;
-  protected final BridgeServerAdvisor advisor;
+  protected final CacheServerAdvisor advisor;
   protected ServerLocation location;
   private final PollingThread pollingThread;
   protected volatile ServerLoad lastLoad;
   protected CacheServerStats stats;
 
   public LoadMonitor(ServerLoadProbe probe, int maxConnections,
-      long pollInterval, int forceUpdateFrequency, BridgeServerAdvisor advisor) {
+      long pollInterval, int forceUpdateFrequency, CacheServerAdvisor advisor) {
     this.probe = probe;
     this.metrics = new ServerMetricsImpl(maxConnections);
     this.pollingThread = new PollingThread(pollInterval, forceUpdateFrequency);
@@ -195,8 +195,8 @@ public class LoadMonitor implements ConnectionListener {
             
             stats.setLoad(load);
             if (locators != null) {
-              BridgeServerLoadMessage message =
-                new BridgeServerLoadMessage(load, location, myClientIds);
+              CacheServerLoadMessage message =
+                new CacheServerLoadMessage(load, location, myClientIds);
               message.setRecipients(locators);
               MembershipManager mgr = advisor.getDistributionManager().getMembershipManager();
               if (mgr == null || !mgr.isBeingSick()) { // test hook

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeClient.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeClient.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeClient.java
deleted file mode 100644
index 184e8bc..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeClient.java
+++ /dev/null
@@ -1,156 +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.cache.util;
-
-import java.util.Properties;
-
-import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.CacheLoader;
-import com.gemstone.gemfire.cache.CacheLoaderException;
-import com.gemstone.gemfire.cache.CacheWriter;
-import com.gemstone.gemfire.cache.LoaderHelper;
-import com.gemstone.gemfire.cache.Region;
-
-/**
- * This class combines the BridgeWriter and BridgeLoader functionality into one
- * class, sharing BridgeServer connections, optimizing the number
- * of connections required when using a BridgeWriter and BridgeLoader separately.
- * <p>
- * When a BridgeClient is declared in cache.xml
- * it can be installed as either a cache-loader or as a cache-writer
- * and it will automatically be installed as both the loader and writer
- * for that region.
- * This allows a single instance to be declared in XML and used as both
- * the cache-loader and cache-writer thus reducing the number of connections to the server.
- * 
- * <p>
- * For configuration details please see the {@link com.gemstone.gemfire.cache.util.BridgeWriter} and 
- * the {@link com.gemstone.gemfire.cache.util.BridgeLoader}.
- * 
- * @author Mitch Thomas
- * @since 5.0.1
- * @see com.gemstone.gemfire.cache.util.BridgeLoader
- * @see com.gemstone.gemfire.cache.util.BridgeWriter
- * @see com.gemstone.gemfire.cache.util.BridgeServer
- * @deprecated as of 5.7 use {@link com.gemstone.gemfire.cache.client pools} instead.
- */
-@Deprecated
-public class BridgeClient extends BridgeWriter implements CacheLoader
-{
-
-  private final BridgeLoader loader = new BridgeLoader();
-  
-  public Object load(LoaderHelper helper) throws CacheLoaderException
-  {
-    return this.loader.load(helper);
-  }
-
-  /**
-   * Ensure that the BridgeLoader class gets loaded.
-   * 
-   * @see SystemFailure#loadEmergencyClasses()
-   */
-  public static void loadEmergencyClasses() {
-    BridgeLoader.loadEmergencyClasses();
-  }
-  
-  @Override
-  public void close()
-  {
-    try {
-      this.loader.close();
-    } finally {
-      super.close();
-    }
-  }
-
-  /**
-   * Returns true if this <code>BridgeClient</code> has been closed.
-   */
-  @Override
-  public boolean isClosed() {
-    return super.isClosed();
-  }
-
-  /**
-   * Notify the BridgeClient that the given Region will begin delivering events to this BridgeClient.
-   * This method effects the behavior of {@link #close()} and allows a single instance of BridgeClient 
-   * to be safely shared with multiple Regions.
-   *
-   * This is called internally when the BridgeClient is added to a Region
-   * via {@link AttributesFactory#setCacheWriter(CacheWriter)}}
-   *
-   * @param r
-   *          the Region which will begin use this BridgeWriter.
-   *
-   * @see #detach(Region)
-   * @see #close()
-   */
-  @Override
-  public void attach(Region r)
-  {
-    try {
-      this.loader.attach(r);
-    } finally {
-      super.attach(r);
-    }
-  }
-
-  /**
-   * Notify the BridgeClient that the given region is no longer relevant.
-   * This method is used internally during Region {@link Region#destroyRegion() destruction} and {@link Region#close() closure}.
-   * This method effects the behavor of {@link #close()} and allows a single instance of BridgeClient 
-   * to be safely shared with multiple Regions.
-   *
-   * @see #attach(Region)
-   * @see #close()
-   * @param r
-   *          the Region which will no longer use this BridgeWriter
-   */
-  @Override
-  public void detach(Region r)
-  {
-    try {
-      this.loader.detach(r);
-    } finally {
-      super.detach(r);
-    }
-  }
-
-  @Override
-  public void init(BridgeWriter bridgeWriter)
-  {
-    super.init(bridgeWriter);
-    this.loader.init(this);
-  }
-
-  @Override
-  public void init(Properties p)
-  {
-    super.init(p);
-    this.loader.init(this);
-  }
-
-  /**
-   * Return the internally maintained BridgeLoader 
-   * @return the internal BridgeLoader
-   */
-  public BridgeLoader getBridgeLoader() {
-    return this.loader;
-  }
-  
-  /**
-   * Returns a string description of the BridgeClient
-   */
-  @Override
-  public String toString()
-  {
-    return "BridgeClient#" + System.identityHashCode(this) +  " connected to " + this.proxy;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeLoader.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeLoader.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeLoader.java
deleted file mode 100644
index 58c5862..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeLoader.java
+++ /dev/null
@@ -1,607 +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.cache.util;
-
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.AttributesMutator;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheLoader;
-import com.gemstone.gemfire.cache.CacheLoaderException;
-import com.gemstone.gemfire.cache.Declarable;
-import com.gemstone.gemfire.cache.LoaderHelper;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.client.internal.BridgePoolImpl;
-import com.gemstone.gemfire.cache.client.internal.ServerProxy;
-import com.gemstone.gemfire.cache.query.SelectResults;
-import com.gemstone.gemfire.internal.cache.tier.ConnectionProxy;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-
-/**
- * A <code>CacheLoader</code> that loads data from one or more remote
- * <code>cacheserver</code> processes. This allows for a hierarchical caching
- * scheme in which one cache ('client' cache) delegates a request to another
- * cache ('server' cache) when it cannot find the data locally.
- * 
- * 
- * When using the <code>BridgeLoader</code>, at least two GemFire Caches must
- * be running in a client/server mode (they should not be part of the same
- * distributed system).
- * 
- * The 'server' cache must be running a gemfire <code>cacheserver</code>
- * process, while the 'client' cache must have a <code>BridgeLoader</code>
- * installed in one or more of its <code>Regions</code>. If a
- * <code>BridgeLoader</code> is defined in a client <code>Region</code>,
- * there must also be a <code>Region</code> defined in the 'server' cache with
- * the same exact name.
- * 
- * <p>
- * 
- * The <code>BridgeLoader</code> performs <code>get()</code> operations on
- * the remote server cache, and does not provide the distribution behavior that
- * can be enabled by using a <code>DISTRIBUTED</code> or
- * <code>DISTRIBUTED_NO_ACK</code> <code>Region</code>. This mechanism is
- * designed as a more targeted alternative to netSearch, in which the 'client'
- * cache completely delegates the loading of the data to the 'server' cache if
- * it is not yet cached in the client. This directed behavior enables a remote
- * network <code>get()</code> operation to be performed much more efficiently
- * in a scenario where there is a hierarchical cache topology. Updates and
- * invalidation remain local, in fact the <code>Regions</code> that are used
- * for this loosely coupled cache may even be <code>LOCAL</code> in scope.
- * 
- * The <code>BridgeLoader</code> may be used to configure caches with
- * multi-layer hierarchies.
- * 
- * 
- * <p>
- * <b>Load Balancing: </b>
- * <p>
- * The <code>BridgeLoader</code> supports these load balancing mechanisms
- * (specified by the <code>LBPolicy</code> config attribute):
- * <p>
- * <ul>
- * <li><b>Sticky </b> <br>
- * In this mode, the client loader picks the first server from the list of
- * servers and establishes a connection to it. Once this connection has been
- * established, every request from that particular 'client' cache is sent on
- * that connection. If requests time out or produce exceptions, the
- * <code>BridgeLoader</code> picks another server and then sends further
- * requests to that server. This achieves a level of load balancing by
- * redirecting requests away from servers that produce timeouts.</li>
- *
- * <li><b>RandomSticky </b> <br>
- * The behavior is the same as Sticky, however the initial assignment of the
- * connection is randomly selected from the list of servers.</li>
- * 
- * <li><b>RoundRobin </b> <br>
- * In this mode, the client establishes connections to all the servers in the
- * server list and then randomly picks a server for each given request. For the
- * next request, it picks the next server in the list.</li>
- * 
- * <li><b>Random </b>: <br>
- * In this mode, the edge establishes connections to all the servers in the
- * server list and then randomly picks a server for every request.</li>
- * 
- * 
- * </ul>
- * 
- * <p>
- * <b>Failover: </b>
- * <p>
- * 
- * If a remote server cache throws an exception or times out, the client will
- * retry based on the configured <code>retryCount</code> parameter. If the
- * <code>retryCount</code> is exceeded, the server in question will be added
- * to a failed server list, and the client will select another server to connect
- * to. The servers in the failed server list will be periodically pinged with an
- * intelligent ping that ensures cache health. If a server is determined to be
- * healthy again, it will be promoted back to the healthy server list. The time
- * period between failed server pings is configurable via the
- * <code>retryInterval</code> parameter.
- * 
- * <p>
- * <b>Configuration: </b>
- * <p>
- * The <code>BridgeLoader</code> is configurable declaratively or
- * programmatically. Declarative configuration is achieved through defining the
- * configuration parameters in a <code>cache.xml</code> file. Programmatic
- * configuration may be achieved by first instantiating a
- * <code>BridgeLoader</code> object and subsequently calling
- * {@link #init(Properties)}with a <code>Properties</code> object containing
- * each desired parameter and value.
- * <p>
- * <b>The supported parameters are: </b>
- * <p>
- * <ul>
- * <li><b>endpoints </b> (required) <br>
- * A comma delimited list of logical names, hostnames, and ports of 'server'
- * caches to connect to <br>
- * The endpoints parameter follows this syntax:
- * logicalName=host:port,logicalName2=host2:port2,.... <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;endpoints&quot;&gt;
- *   &lt;string&gt;MyPrimaryServer=hostsrv:40404,MySecondary=hostsrv2:40404&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>readTimeout </b> (optional: default 10000) <br>
- * A millisecond value representing the amount of time to wait for a response
- * from a cache server. <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;readTimeout&quot;&gt;
- *   &lt;string&gt;5000&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- *  </li>
- *  
- * <li><b>retryAttempts </b> (optional: default 5)<br>
- * The number of times to retry a request after timeout/exception. <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;retryAttempts&quot;&gt;
- *   &lt;string&gt;5&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>retryInterval </b> (optional: default 10000) <br>
- * A millisecond value representing the amount of time to wait between attempts
- * by the <code>ServerMonitor</code> to ping living servers to verify that
- * they are still alive and dead servers to verify that they are still dead.
- * <br>
- * Example:</li>
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;retryInterval&quot;&gt;
- *   &lt;string&gt;10000&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * <li><b>LBPolicy </b> (optional: default "Sticky") <br>
- * A String value representing the load balancing policy to use. See above for
- * more details. <br>
- * Options are:
- * <ul>
- * <li>"Sticky"</li>
- * <li>"RandomSticky"</li>
- * <li>"RoundRobin"</li>
- * <li>"Random"</li>
- * </ul>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;LBPolicy&quot;&gt;
- *   &lt;string&gt;Sticky&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>connectionsPerServer </b> (optional: default 1)<br>
- * The number of initial connections created to each time it is
- * determined to be alive.
- * The minimum of <code>0</code> causes no initial connections to be created (they are only created on demand).
- * <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;connectionsPerServer&quot;&gt;
- *   &lt;string&gt;10&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * <li><b>socketBufferSize </b> (optional: default 32768) <br>
- * The size of the socket buffers in bytes. <br>
- * Example:
- * 
- * <pre>
- *<code>
- * &lt;parameter name=&quot;socketBufferSize&quot;&gt;
- *   &lt;string&gt;32768&lt;/string&gt;
- * &lt;/parameter&gt;
- * </code>
- *</pre>
- * 
- * </li>
- * </ul>
- * 
- * <p>
- * 
- * If you are using a <code>cache.xml</code> file to create a
- * <code>Region</code> declaratively, you can include the following
- * &lt;cache-loader&gt; definition to associate a <code>BridgeLoader</code>
- * with a <code>Region</code> (default values shown for optional parameters):
- * 
- * <pre>
- * 
- * &lt;cache-loader&gt;
- *   &lt;class-name&gt;com.gemstone.gemfire.cache.util.BridgeLoader&lt;/class-name&gt;
- *   &lt;parameter name=&quot;endpoints&quot;&gt;
- *     &lt;string&gt;MyHost=ninja.gemstone.com:40404&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;readTimeout&quot;&gt;
- *     &lt;string&gt;10000&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;retryAttempts&quot;&gt;
- *     &lt;string&gt;5&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;retryInterval&quot;&gt;
- *     &lt;string&gt;10000&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;LBPolicy&quot;&gt;
- *     &lt;string&gt;Sticky&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;parameter name=&quot;socketBufferSize&quot;&gt;
- *     &lt;string&gt;32768&lt;/string&gt;
- *   &lt;/parameter&gt;
- *   &lt;/parameter&gt;
- * &lt;/cache-loader&gt;
- * </pre>
- * 
- * @since 2.0.2
- * @author Greg Passmore
- * @deprecated as of 5.7 use {@link com.gemstone.gemfire.cache.client pools} instead.
- */
-
-@Deprecated
-public class BridgeLoader implements CacheLoader, Declarable
-{
-  ConnectionProxy proxy = null; // package access for
-                                    // tests/com/gemstone/gemfire/cache/util/BridgeHelper
-
-  private Properties properties;
-  
-  public BridgeLoader() { }
-
-  /**
-   * Creates a loader from an existing <code>BridgeWriter</code>. This
-   * method reuses the existing <code>BridgeWriter</code>'s proxy.
-   * 
-   * @param bw
-   *          The existing <code>BridgeWriter</code>
-   * 
-   * @since 5.7
-   */
-  public BridgeLoader(BridgeWriter bw) {
-    init(bw);
-  }
-
-  private volatile boolean isClosed = false;
-
-  private final AtomicInteger refCount = new AtomicInteger();
-
-  /**
-   * Initializes the loader with supplied config parameters. If instantiating
-   * the loader programmatically, this method must be called with a
-   * <code>Properties</code> object that at a minimum contains the 'endpoints'
-   * parameter before the loader can be used. If a LicenseException is thrown
-   * during initialization the BridgeLoader will trhow IllegalStateExceptions
-   * until properly initialized.
-   * 
-   * @param p
-   *          configuration data such as 'endpoint' definitions
-   * @throws IllegalStateException if the loader is already initialized
-   */
-  public void init(Properties p)
-  {
-    if (this.proxy != null) throw new IllegalStateException(LocalizedStrings.BridgeLoader_ALREADY_INITIALIZED.toLocalizedString());
-    this.properties = p;
-    if (Boolean.getBoolean("skipConnection")) {
-      // used by hydra when generating XML via RegionAttributesCreation
-      return;
-    }
-    this.proxy = BridgePoolImpl.create(properties, false/*useByBridgeWriter*/);
-  }
-
-  /**
-   * Initializes this loader from an existing <code>BridgeLoader</code>.
-   * This method reuses the existing <code>BridgeLoader</code>'s connections
-   * to the server.
-   *
-   * @param bridgeLoader The existing <code>BridgeLoader</code>
-   * @throws IllegalStateException if the loader is already initialized
-   *
-   * @since 4.2
-   */
-  public void init(BridgeLoader bridgeLoader)
-  {
-    if (this.proxy != null) throw new IllegalStateException(LocalizedStrings.BridgeLoader_ALREADY_INITIALIZED.toLocalizedString());
-    ConnectionProxy p = bridgeLoader.proxy;
-    p.reuse();
-    this.proxy = p;
-  }
-
-  /**
-   * Initializes this loader from an existing <code>BridgeWriter</code>. This
-   * method reuses the existing <code>BridgeWriter</code>'s proxy.
-   * 
-   * @param bridgeWriter
-   *          The existing <code>BridgeWriter</code>
-   * @throws IllegalStateException if the loader is already initialized
-   * 
-   * @since 5.7
-   */
-  public void init(BridgeWriter bridgeWriter)
-  {
-    if (this.proxy != null) throw new IllegalStateException("Already initialized");
-    ConnectionProxy p = bridgeWriter.proxy;
-    p.reuse();
-    this.proxy = p;
-  }
-
-  /**
-   * Ensure that the ConnectionProxyImpl class gets loaded.
-   * 
-   * @see SystemFailure#loadEmergencyClasses()
-   */
-  public static void loadEmergencyClasses() {
-    BridgePoolImpl.loadEmergencyClasses();
-  }
-  
-  /**
-   * Called when the region containing this <code>BridgeLoader</code> is
-   * destroyed, when the {@link Cache}is closed, or when a callback is removed
-   * from a region using an {@link AttributesMutator}
-   *
-   * Closes connections to {@link BridgeServer}s when all {@link Region}s are
-   * finished using this BridgeLoader,
-   *
-   * @see #detach(Region)
-   * @see #attach(Region)
-   */
-  public void close() {
-    if (this.refCount.get() <= 0) {
-      this.isClosed = true;
-      proxy.close();
-    }
-  }
-
-  /**
-   * Returns true if this <code>BridgeLoader</code> has been closed.
-   */
-  public boolean isClosed() {
-    return this.isClosed;
-  }
-
-  /**
-   * For speed optimizations, a connection to a server may be assigned to the
-   * calling thread when load is called. When the application thread is done
-   * doing its work it should invoke the BridgeLoader close method. This frees
-   * up the connection assigned to the application thread.
-   */
-  public void release()
-  {
-    proxy.release();
-  }
-
-  /**
-   * This method should be invoked when the BridgeLoader mechanism is to be shut
-   * down explicitly , outside of closing the cache.
-   */
-  public void terminate()
-  {
-    this.isClosed = true;
-    proxy.terminate();
-  }
-
-  // removed checkForTransaction
-
-  /**
-   * This method is invoked implicitly when an object requested on the client
-   * cache cannot be found. The server cache will attempt to be contacted, and
-   * if no server cache is available (or healthy) a CacheLoaderException will
-   * be thrown.
-   * 
-   */
-  public Object load(LoaderHelper helper) throws CacheLoaderException
-  {
-    throw new IllegalStateException("this method should not be called"); 
-  }
-
-  private void checkClosed() {
-    String reason = this.proxy.getCancelCriterion().cancelInProgress();
-    if(reason != null) {
-      throw new BridgeWriterException("The BridgeWriter has been closed: " + reason);
-    }
-    
-    if (this.isClosed) {
-      throw new CacheLoaderException(LocalizedStrings.BridgeLoader_THE_BRIDGELOADER_HAS_BEEN_CLOSED.toLocalizedString());
-    }
-    if (this.proxy != null && !this.proxy.isOpen()) {
-      throw new CacheLoaderException(LocalizedStrings.BridgeLoader_THE_BRIDGELOADER_HAS_BEEN_CLOSED.toLocalizedString());
-    }
-  }
-
-  /**
-   * Invoke a query on the cache server.
-   * @deprecated use {@link Region#query} instead
-   */
-  @Deprecated
-  public SelectResults query(String queryStr) throws CacheLoaderException {
-    ServerProxy sp = new ServerProxy((BridgePoolImpl)this.proxy);
-    return sp.query(queryStr, null);
-  }
-
-  /**
-   * Returns the retry interval in use. Retry interval refers to the interval at
-   * which dead servers are attempted to be reconnected. Internal use only.
-   */
-  public int getRetryInterval()
-  {
-    return proxy.getRetryInterval();
-  }
-
-  /**
-   * Returns the read timeout being used to time out requests to the server
-   * Internal use only.
-   */
-  public int getReadTimeout()
-  {
-    return proxy.getReadTimeout();
-  }
-
-  /**
-   * Returns the number of times the bridge loader tries to get data on
-   * encountering certain types of exceptions. Internal use only
-   */
-  public int getRetryAttempts()
-  {
-    return this.proxy.getRetryAttempts();
-  }
-
-  /**
-   * Returns the load balancing policy being used by the bridge loader Internal
-   * use only
-   */
-  public String getLBPolicy()
-  {
-    return proxy.getLBPolicy();
-  }
-
-  /**
-   * Returns the properties that defined this <code>BridgeWriter</code>.
-   * 
-   * @return the properties that defined this <code>BridgeWriter</code>
-   * 
-   * @since 4.2
-   */
-  public Properties getProperties()
-  {
-    return this.properties;
-  }
-
-  /**
-   * Returns the <code>ConnectionProxy</code> associated with this
-   * <code>BridgeLoader</code>.
-   * 
-   * For internal use only.
-   * 
-   * @return the <code>ConnectionProxy</code> associated with this
-   *         <code>BridgeLoader</code>
-   */
-  public Object/*ConnectionProxy*/ getConnectionProxy()
-  {
-    return proxy;
-  }
-
-  /**
-   * Add an <code>Endpoint</code> to the known <code>Endpoint</code>s.
-   * 
-   * @param name The name of the endpoint to add
-   * @param host The host name or ip address of the endpoint to add
-   * @param port The port of the endpoint to add
-   * 
-   * @throws EndpointExistsException if the <code>Endpoint</code> to be
-   * added already exists.
-   * 
-   * @since 5.0.2
-   */
-  public void addEndpoint(String name, String host, int port)
-  throws EndpointExistsException {
-    this.proxy.addEndpoint(name, host, port);
-  }
-
-  /**
-   * Remove an <code>Endpoint</code> from the dead <code>Endpoint</code>s.
-   * The specified <code>Endpoint</code> must be dead.
-   * 
-   * @param name The name of the endpoint to remove
-   * @param host The host name or ip address of the endpoint to remove
-   * @param port The port of the endpoint to remove
-   * 
-   * @throws EndpointDoesNotExistException if the <code>Endpoint</code> to be
-   * removed doesn't exist.
-   * 
-   * @throws EndpointInUseException if the <code>Endpoint</code> to be removed
-   * contains <code>Connection</code>s
-   * 
-   * @since 5.0.2
-   */
-  public void removeEndpoint(String name, String host, int port)
-  throws EndpointDoesNotExistException, EndpointInUseException {
-    this.proxy.removeEndpoint(name, host, port);
-  }
-
-  // removed handleException
-
-  // removed getExceptionMessage
-
-  /**
-   * Returns a brief description of this <code>BridgeLoader</code>
-   * 
-   * @since 4.0
-   */
-  @Override
-  public String toString()
-  {
-    return LocalizedStrings.BridgeLoader_BRIDGELOADER_CONNECTED_TO_0.toLocalizedString(this.proxy);
-  }
-
-  /**
-   * Notify the BridgeLoader that the given region is no longer relevant. This
-   * method is used internally during Region
-   * {@link Region#destroyRegion() destruction}and
-   * {@link Region#close() closure}. This method effects the behavor of
-   * {@link #close()}.
-   *
-   * @param r
-   *          the Region which will no longer use this BridgeLoader
-   * @see #close()
-   * @see #attach(Region)
-   * @since 4.3
-   */
-  public void detach(Region r)
-  {
-    this.refCount.decrementAndGet();
-  }
-
-  /**
-   * Notify the BridgeLoader that the given Region will begin calling
-   * {@link #load(LoaderHelper)}.
-   *
-   * This method affects the behavior of {@link #close()}.
-   *
-   * This is called internally when the BridgeLoader is added to a Region via
-   * {@link AttributesFactory#setCacheLoader(CacheLoader)}
-   * 
-   * @param r
-   *          the Region which will begin use this BridgeWriter.
-   * @since 4.3
-   *
-   * @see #detach(Region)
-   * @see #close()
-   */
-  public void attach(Region r)
-  {
-    checkClosed();
-    this.refCount.incrementAndGet();
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembership.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembership.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembership.java
deleted file mode 100755
index fdb718d..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembership.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
-
-/**
- * Provides utility methods for registering and unregistering
- * BridgeMembershipListeners in this process.
- *
- * @author Kirk Lund
- * @since 4.2.1
- * @deprecated see com.gemstone.gemfire.management.membership.ClientMembership
- */
-public final class BridgeMembership {
-
-  private BridgeMembership() {}
-
-  /**
-   * Registers a {@link BridgeMembershipListener} for notification of
-   * connection changes for BridgeServers and bridge clients.
-   * @param listener a BridgeMembershipListener to be registered
-   */
-  public static void registerBridgeMembershipListener(BridgeMembershipListener listener) {
-    InternalBridgeMembership.registerBridgeMembershipListener(listener);
-  }
-
-  /**
-   * Removes registration of a previously registered {@link
-   * BridgeMembershipListener}.
-   * @param listener a BridgeMembershipListener to be unregistered
-   */
-  public static void unregisterBridgeMembershipListener(BridgeMembershipListener listener) {
-    InternalBridgeMembership.unregisterBridgeMembershipListener(listener);
-  }
-
-  /**
-   * Returns an array of all the currently registered
-   * <code>BridgeMembershipListener</code>s. Modifications to the returned
-   * array will not effect the registration of these listeners.
-   * @return the registered <code>BridgeMembershipListener</code>s; an empty
-   * array if no listeners
-   */
-  public static BridgeMembershipListener[] getBridgeMembershipListeners() {
-    return InternalBridgeMembership.getBridgeMembershipListeners();
-  }
-
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipEvent.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipEvent.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipEvent.java
deleted file mode 100755
index b9d205e..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipEvent.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-package com.gemstone.gemfire.cache.util;
-
-import com.gemstone.gemfire.management.membership.ClientMembershipEvent;
-
-/**
- * An event delivered to a {@link BridgeMembershipListener} when this 
- * process detects connection changes to BridgeServers or bridge clients.
- *
- * @author Kirk Lund
- * @since 4.2.1
- * @deprecated see com.gemstone.gemfire.management.membership.ClientMembershipEvent
- */
-public interface BridgeMembershipEvent extends ClientMembershipEvent {
-  
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipListener.java
deleted file mode 100755
index d0e6196..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipListener.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
- package com.gemstone.gemfire.cache.util;
-
-/**
- * A listener whose callback methods are invoked when this process 
- * detects connection changes to BridgeServers or bridge clients.
- *
- * @see BridgeMembership#registerBridgeMembershipListener
- *
- * @author Kirk Lund
- * @since 4.2.1
- * @deprecated see com.gemstone.gemfire.management.membership.ClientMembershipListener
- */
-public interface BridgeMembershipListener{
-
-  /**
-   * Invoked when a client has connected to this process or when this
-   * process has connected to a BridgeServer.
-   */
-  public void memberJoined(BridgeMembershipEvent event);
-
-  /**
-   * Invoked when a client has gracefully disconnected from this process
-   * or when this process has gracefully disconnected from a BridgeServer.
-   */
-  public void memberLeft(BridgeMembershipEvent event);
-
-  /**
-   * Invoked when a client has unexpectedly disconnected from this process
-   * or when this process has unexpectedly disconnected from a BridgeServer.
-   */
-  public void memberCrashed(BridgeMembershipEvent event);
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipListenerAdapter.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipListenerAdapter.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipListenerAdapter.java
deleted file mode 100755
index 3ce2721..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeMembershipListenerAdapter.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-
-package com.gemstone.gemfire.cache.util;
-
-/**
- * Utility class that implements all methods in 
- * <code>BridgeMembershipListener</code> with empty implementations.
- * Applications can subclass this class and only override the methods for
- * the events of interest.
- *
- * @author Kirk Lund
- * @since 4.2.1
- * @deprecated see com.gemstone.gemfire.management.membership.ClientMembershipListenerAdapter
- */
-public abstract class BridgeMembershipListenerAdapter 
-implements BridgeMembershipListener {
-    
-  /**
-   * Invoked when a client has connected to this process or when this
-   * process has connected to a BridgeServer.
-   */
-  public void memberJoined(BridgeMembershipEvent event) {}
-
-  /**
-   * Invoked when a client has gracefully disconnected from this process
-   * or when this process has gracefully disconnected from a BridgeServer.
-   */
-  public void memberLeft(BridgeMembershipEvent event) {}
-
-  /**
-   * Invoked when a client has unexpectedly disconnected from this process
-   * or when this process has unexpectedly disconnected from a BridgeServer.
-   */
-   public void memberCrashed(BridgeMembershipEvent event) {}
-  
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeServer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeServer.java
deleted file mode 100644
index ad77071..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/util/BridgeServer.java
+++ /dev/null
@@ -1,442 +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.cache.util;
-
-import java.io.IOException;
-import java.util.Set;
-
-import com.gemstone.gemfire.cache.ClientSession;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.cache.client.Pool;
-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.server.internal.ConnectionCountProbe;
-
-/**
- * A cache bridge server that serves the contents of a
- * <code>Cache</code> to VMs in another distributed system via a
- * socket.  A bridge server is used in conjunction with a
- * client {@link Pool} to connect two regions
- * that reside in different distributed systems.
- *
- * @see com.gemstone.gemfire.cache.Cache#addBridgeServer
- * @see com.gemstone.gemfire.cache.Cache#getBridgeServers
- *
- * @since 2.0.2
- * @deprecated as of 5.7 used {@link CacheServer} instead.
- */
-@Deprecated
-public interface BridgeServer {
-
-  /** The default port on which a <Code>BridgeServer</code> is
-   * configured to serve. */
-  public static final int DEFAULT_PORT = 40404;
-
-  /** 
-   * The default number of sockets accepted by a BridgeServer. 
-   * When the maximum is reached the server will stop accepting new connections.
-   * Current value: 800
-   * @since 5.7
-   */
-  public static final int DEFAULT_MAX_CONNECTIONS = 800;
-  // Value derived from common file descriptor limits for Unix sytems (1024)... 
-
-  /** 
-   * The default limit to the maximum number of server threads that can be
-   * created to service client requests. Once this number of threads exist then
-   * connections must share the same thread to service their request. A selector
-   * is used to detect client connection requests and dispatch them to the thread
-   * pool.
-   * The default of <code>0</code> causes a thread to be bound to every connection
-   * and to be dedicated to detecting client requests on that connection. A selector
-   * is not used in this default mode.
-   * Current value: 0
-   * @since 5.7
-   */
-  public static final int DEFAULT_MAX_THREADS = 0;
-
-  /** The default notify-by-subscription value which tells the
-   * <Code>BridgeServer</code> whether or not to notify clients
-   * based on key subscription. */
-  public static final boolean DEFAULT_NOTIFY_BY_SUBSCRIPTION = true;
-
-  /**
-   * The default socket buffer size for socket buffers from the server
-   * to the client.
-   */
-  public static final int DEFAULT_SOCKET_BUFFER_SIZE = 32768;
-
-  /**
-   * The default maximum amount of time between client pings. This value
-   * is used by the <code>ClientHealthMonitor</code> to determine the
-   * health of this <code>BridgeServer</code>'s clients.
-   */
-  public static final int DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS = 60000;
-
-  /**
-   * The default maximum number of messages that can be enqueued in a
-   * client-queue.
-   */
-  public static final int DEFAULT_MAXIMUM_MESSAGE_COUNT = 230000;
-
-  /**
-   * The default time (in seconds ) after which a message in the client queue
-   * will expire.
-   */
-  public static final int DEFAULT_MESSAGE_TIME_TO_LIVE = 180;
-
-  /**
-   * The default list of server groups a server belongs to.
-   * The current default is an empty list.
-   * @since 5.7
-   */
-  public static final String[] DEFAULT_GROUPS = new String[0];
-  
-  /**
-   * The default load balancing probe. The default load balancing
-   * probe reports the connections counts of this server. 
-   * @since 5.7
-   *  
-   */
-  public static final ServerLoadProbe DEFAULT_LOAD_PROBE = new ConnectionCountProbe();
-  
-  /**
-   * The default frequency at which to poll the load probe for the load
-   * on this server. Defaults to 5000 (5 seconds).
-   * @since 5.7
-   */
-  public static final long DEFAULT_LOAD_POLL_INTERVAL = 5000;
-
-  /**
-   * The default ip address or host name that the server's socket will
-   * listen on for client connections.
-   * The current default is an empty string.
-   * @since 5.7
-   */
-  public static final String DEFAULT_BIND_ADDRESS = "";
-
-  /**
-   * The default ip address or host name that will be given to clients
-   * as the host this server is listening on.
-   * The current default is an empty string.
-   * @since 5.7
-   */
-  public static final String DEFAULT_HOSTNAME_FOR_CLIENTS = "";
-
-  /**
-   * Returns the port on which this server listens for clients.
-   */
-  public int getPort();
-
-  /**
-   * Sets the port on which this server listens for clients.
-   *
-   * @throws IllegalStateException
-   *         If this server is running
-   */
-  public void setPort(int port);
-
-  /**
-   * Returns a string representing the ip address or host name that this server
-   * will listen on.
-   * @return the ip address or host name that this server is to listen on
-   * @see #DEFAULT_BIND_ADDRESS
-   * @since 5.7
-   */
-  public String getBindAddress();
-  /**
-   * Sets the ip address or host name that this server is to listen on for
-   * client connections.
-   * <p>Setting a specific bind address will cause the server to always
-   * use this address and ignore any address specified by "server-bind-address"
-   * or "bind-address" in the <code>gemfire.properties</code> file
-   * (see {@link com.gemstone.gemfire.distributed.DistributedSystem}
-   * for a description of these properties).
-   * <p> The value <code>""</code> does not override the <code>gemfire.properties</code>.
-   * It will cause the local machine's default address to be listened on if the
-   * properties file does not specify and address.
-   * If you wish to override the properties and want to have your server bind to all local
-   * addresses then use this bind address <code>"0.0.0.0"</code>.
-   * <p> A <code>null</code> value will be treated the same as the default <code>""</code>.
-   * @param address the ip address or host name that this server is to listen on
-   * @see #DEFAULT_BIND_ADDRESS
-   * @since 5.7
-   */
-  public void setBindAddress(String address);
-  /**
-   * Returns a string representing the ip address or host name that server locators
-   * will tell clients that this server is listening on.
-   * @return the ip address or host name to give to clients so they can connect
-   *         to this server
-   * @see #DEFAULT_HOSTNAME_FOR_CLIENTS
-   * @since 5.7
-   */
-  public String getHostnameForClients();
-  /**
-   * Sets the ip address or host name that this server is to listen on for
-   * client connections.
-   * <p>Setting a specific hostname-for-clients will cause server locators
-   * to use this value when telling clients how to connect to this server.
-   * This is useful in the case where the server may refer to itself with one
-   * hostname, but the clients need to use a different hostname to find the 
-   * server.
-   * <p> The value <code>""</code> causes the <code>bind-address</code> to be given to clients.
-   * <p> A <code>null</code> value will be treated the same as the default <code>""</code>.
-   * @param name the ip address or host name that will be given to clients
-   *   so they can connect to this server
-   * @see #DEFAULT_HOSTNAME_FOR_CLIENTS
-   * @since 5.7
-   */
-  public void setHostnameForClients(String name);
-  /**
-   * Sets whether or not this server should notify clients based on
-   * key subscription.
-   *
-   * If false, then an update to any key on the server causes an update to
-   * be sent to all clients. This update does not push the actual data to the
-   * clients. Instead, it causes the client to locally invalidate or destroy
-   * the corresponding entry. The next time the client requests the key, it
-   * goes to the server for the value.
-   *
-   * If true, then an update to any key on the server causes an update to be
-   * sent to only those clients who have registered interest in that key. Other
-   * clients are not notified of the change. In addition, the actual value is
-   * pushed to the client. The client does not need to request the new value
-   * from the server.
-   *
-   * @since 4.2
-   * @deprecated as of 6.0.1. This method is no more in use, by default 
-   * notifyBySubscription attribute is set to true.
-   */
-  @Deprecated
-  public void setNotifyBySubscription(boolean b);
-
-  /**
-   * Answers whether or not this server should notify clients based on
-   * key subscription.
-   *
-   * @since 4.2
-   * @deprecated as of 6.0.1. This method is no more in use, by default 
-   * notifyBySubscription attribute is set to true.
-   */
-  @Deprecated
-  public boolean getNotifyBySubscription();
-
-  /**
-   * Sets the buffer size in bytes of the socket connection for this
-   * <code>BridgeServer</code>. The default is 32768 bytes.
-   *
-   * @param socketBufferSize The size in bytes of the socket buffer
-   *
-   * @since 4.2.1
-   */
-  public void setSocketBufferSize(int socketBufferSize);
-
-  /**
-   * Returns the configured buffer size of the socket connection for this
-   * <code>BridgeServer</code>. The default is 32768 bytes.
-   * @return the configured buffer size of the socket connection for this
-   * <code>BridgeServer</code>
-   *
-   * @since 4.2.1
-   */
-  public int getSocketBufferSize();
-
-  /**
-   * Sets the maximum amount of time between client pings. This value is
-   * used by the <code>ClientHealthMonitor</code> to determine the health
-   * of this <code>BridgeServer</code>'s clients. The default is 60000 ms.
-   *
-   * @param maximumTimeBetweenPings The maximum amount of time between client
-   * pings
-   *
-   * @since 4.2.3
-   */
-  public void setMaximumTimeBetweenPings(int maximumTimeBetweenPings);
-
-  /**
-   * Returns the maximum amount of time between client pings. This value is
-   * used by the <code>ClientHealthMonitor</code> to determine the health
-   * of this <code>BridgeServer</code>'s clients. The default is 60000 ms.
-   * @return the maximum amount of time between client pings.
-   *
-   * @since 4.2.3
-   */
-  public int getMaximumTimeBetweenPings();
-
-  /**
-   * Starts this server.  Once the server is running, its
-   * configuration cannot be changed.
-   *
-   * @throws IOException
-   *         If an error occurs while starting the server
-   */
-  public void start() throws IOException;
-
-  /**
-   * Returns whether or not this server is running
-   */
-  public boolean isRunning();
-
-  /**
-   * Stops this server.  Note that the
-   * <code>BridgeServer</code> can be reconfigured and restarted if
-   * desired.
-   */
-  public void stop();
-
-  /** 
-   *  Returns the maximum allowed client connections
-   */
-  public int getMaxConnections();
-
-  /**
-   * Sets the maxium number of client connections allowed.
-   * When the maximum is reached the server will stop accepting
-   * connections.
-   * 
-   * @see #DEFAULT_MAX_CONNECTIONS
-   */
-  public void setMaxConnections(int maxCons);
-
-  /** 
-   * Returns the maxium number of threads allowed in this server to service
-   * client requests.
-   * The default of <code>0</code> causes the server to dedicate a thread for
-   * every client connection.
-   * @since 5.1
-   */
-  public int getMaxThreads();
-
-  /**
-   * Sets the maxium number of threads allowed in this server to service
-   * client requests.
-   * The default of <code>0</code> causes the server to dedicate a thread for
-   * every client connection.
-   * 
-   * @see #DEFAULT_MAX_THREADS
-   * @since 5.1
-   */
-  public void setMaxThreads(int maxThreads);
-
-  /**
-   * Returns the maximum number of messages that can be enqueued in a
-   * client-queue.
-   */
-  public int getMaximumMessageCount();
-
-  /**
-   * Sets maximum number of messages that can be enqueued in a client-queue.
-   * 
-   * @see #DEFAULT_MAXIMUM_MESSAGE_COUNT
-   */
-  public void setMaximumMessageCount(int maxMessageCount);
-  
-  /**
-   * Returns the time (in seconds ) after which a message in the client queue
-   * will expire.
-   */
-  public int getMessageTimeToLive();
-
-  /**
-   * Sets the time (in seconds ) after which a message in the client queue
-   * will expire.Expiry settings are applicable for the secondary queues only
-   * This setting has no impact on the primary queue.
-   * 
-   * @see #DEFAULT_MESSAGE_TIME_TO_LIVE
-   */
-  public void setMessageTimeToLive(int messageTimeToLive);
-  
-  /**
-   * Returns the <code>ClientSession</code> associated with the
-   * <code>DistributedMember</code>
-   * @return the <code>ClientSession</code> associated with the
-   * <code>DistributedMember</code>
-   * @since 5.6
-   */
-  public ClientSession getClientSession(DistributedMember member);
-  
-  /**
-   * Returns the <code>ClientSession</code> associated with the
-   * durable client id
-   * @return the <code>ClientSession</code> associated with the
-   * durable
-   * @since 5.6
-   */
-  public ClientSession getClientSession(String durableClientId);
-  
-  /**
-   * Returns a set of all <code>ClientSession</code>s
-   * @return a set of all <code>ClientSession</code>s
-   */
-  public Set getAllClientSessions();
-
-  /**
-   * Sets the list of server groups this cache server will belong to.
-   * By default cache servers belong to the default global server group
-   * which all cache servers always belong to.
-   * @param groups possibly empty array of <code>String</code> where each string
-   * is a server groups that this cache server will be a member of.
-   * @see #DEFAULT_GROUPS
-   * @since 5.7
-   */
-  public void setGroups(String[] groups);
-  /**
-   * Returns the list of server groups that this cache server belongs to.
-   * @return a possibly empty array of <code>String</code>s where
-   * each string is a server group. Modifying this array will not change the
-   * server groups that this cache server belongs to.
-   * @since 5.7
-   */
-  public String[] getGroups();
-
-  /**
-   * Get the load probe for this cache server. See
-   * {@link ServerLoadProbe} for details on the load probe.
-   * @return the load probe used by this cache
-   * server.
-   * @since 5.7
-   */
-  public ServerLoadProbe getLoadProbe();
-
-  /**
-   * Set the load probe for this cache server. See
-   * {@link ServerLoadProbe} for details on how to implement
-   * a load probe.
-   * @param loadProbe the load probe to use for
-   * this cache server.
-   * @since 5.7
-   */
-  public void setLoadProbe(ServerLoadProbe loadProbe);
-
-  /**
-   * Get the frequency in milliseconds to poll the load probe on this cache
-   * server.
-   * 
-   * @return the frequency in milliseconds that we will poll the load probe.
-   */
-  public long getLoadPollInterval();
-
-  /**
-   * Set the frequency in milliseconds to poll the load probe on this cache
-   * server
-   * @param loadPollInterval the frequency in milliseconds to poll
-   * the load probe. Must be greater than 0.
-   */
-  public void setLoadPollInterval(long loadPollInterval);
-  
-  /**
-   * Get the ClientSubscriptionConfig for this cache server. See
-   * {@link ClientSubscriptionConfig} for details on the client subscription configuration.
-   * 
-   * @return ClientSubscriptionConfig
-   * @since 5.7
-   */
-  public ClientSubscriptionConfig getClientSubscriptionConfig();
-}


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java
index fb233c2..6fd9e1d 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java
@@ -31,7 +31,7 @@ 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.query.CqEvent;
-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.cache.util.CqListenerAdapter;
 import com.gemstone.gemfire.distributed.DistributedSystem;
@@ -64,42 +64,16 @@ public class CacheServerTestUtil extends DistributedTestCase
     super(name);
   }
 
-   
-  
-//  public static void createCacheClient(Properties props, String regionName)
-//      throws Exception {
-//    createCacheClient(props, regionName, getClientProperties(), Boolean.FALSE);
-//  }
   public static void createCacheClient(Pool poolAttr, String regionName)
       throws Exception {
     createCacheClient(poolAttr, regionName, getClientProperties(), Boolean.FALSE);
   }
 
-//  public static void createCacheClient(Properties props, String regionName,
-//      Properties dsProperties) throws Exception {
-//    createCacheClient(props, regionName, dsProperties, Boolean.FALSE);
-//  }
   public static void createCacheClient(Pool poolAttr, String regionName,
       Properties dsProperties) throws Exception {
     createCacheClient(poolAttr, regionName, dsProperties, Boolean.FALSE);
   }
 
-//  public static void createCacheClient(Properties props, String regionName,
-//    Properties dsProperties, Boolean addControlListener) throws Exception {
-//    new CacheServerTestUtil("temp").createCache(dsProperties);
-//    BridgeWriter writer = new BridgeWriter();
-//    writer.init(props);
-//    AttributesFactory factory = new AttributesFactory();
-//    factory.setScope(Scope.LOCAL);
-//    factory.setCacheWriter(writer);
-//    if (addControlListener.booleanValue()) {
-//      factory.addCacheListener(new ControlListener());
-//    }
-//    RegionAttributes attrs = factory.create();
-//    cache.createRegion(regionName, attrs);
-//    pool = (PoolImpl)writer.getConnectionProxy();
-//  }
-  
   public static void createClientCache(Pool poolAttr, String regionName) throws Exception {
     createClientCache(poolAttr, regionName, getClientProperties());
   }
@@ -299,7 +273,7 @@ public class CacheServerTestUtil extends DistributedTestCase
   }
 
   /**
-   * Create client regions each with their own BridgeWriter instance.
+   * Create client regions
    * @param props
    * @param regionName1
    * @param regionName2
@@ -356,7 +330,7 @@ public class CacheServerTestUtil extends DistributedTestCase
     factory.setDataPolicy(DataPolicy.REPLICATE);
     RegionAttributes attrs = factory.create();
     cache.createRegion(regionName, attrs);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server1.setPort(port);
     server1.setNotifyBySubscription(notifyBySubscription.booleanValue());
@@ -378,7 +352,7 @@ public class CacheServerTestUtil extends DistributedTestCase
     factory.setDataPolicy(DataPolicy.REPLICATE);
     RegionAttributes attrs = factory.create();
     cache.createRegion(regionName, attrs);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server1.setPort(port);
     server1.setNotifyBySubscription(notifyBySubscription.booleanValue());
@@ -407,7 +381,7 @@ public class CacheServerTestUtil extends DistributedTestCase
     factory.setDataPolicy(DataPolicy.REPLICATE);
     RegionAttributes attrs = factory.create();
     cache.createRegion(regionName, attrs);
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     server.setPort(serverPort.intValue());
     server.setNotifyBySubscription(notifyBySubscription.booleanValue());
     server.start();
@@ -428,7 +402,7 @@ public class CacheServerTestUtil extends DistributedTestCase
     if (!regionName2.equals("")) {
       cache.createRegion(regionName2, attrs);
     }
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server1.setPort(port);
     server1.setNotifyBySubscription(notifyBySubscription.booleanValue());
@@ -504,18 +478,18 @@ public class CacheServerTestUtil extends DistributedTestCase
   }
   
   public static void stopCacheServers() {
-    Iterator iter = getCache().getBridgeServers().iterator();
+    Iterator iter = getCache().getCacheServers().iterator();
     if (iter.hasNext()) {
-      BridgeServer server = (BridgeServer) iter.next();
+      CacheServer server = (CacheServer) iter.next();
       server.stop();
       assertFalse(server.isRunning());
     }
   }
 
   public static void restartCacheServers() {
-    Iterator iter = getCache().getBridgeServers().iterator();
+    Iterator iter = getCache().getCacheServers().iterator();
     if (iter.hasNext()) {
-      BridgeServer server = (BridgeServer) iter.next();
+      CacheServer server = (CacheServer) iter.next();
       try {
         server.start();
       } catch(Exception e) {
@@ -535,11 +509,6 @@ public class CacheServerTestUtil extends DistributedTestCase
     return pool;
   }
 
-//   public static BridgeWriter getWriter()
-//   {
-//     return writer;
-//   }
-
   /**
    * Disables the shuffling of endpoints for a client
    *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java
index 9bb6b59..399175d 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTransactionsDUnitTest.java
@@ -22,7 +22,7 @@ import com.gemstone.gemfire.cache.Region;
 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.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.DistributedSystem;
@@ -772,7 +772,7 @@ public class CacheServerTransactionsDUnitTest extends DistributedTestCase
     });
     Region r1 = cache.createRegion(REGION_NAME, factory.create());
     assertNotNull(r1);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server1.setPort(port);
     server1.setMaxThreads(maxThreads.intValue());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClearPropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClearPropagationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClearPropagationDUnitTest.java
index dad3b7d..ac345e4 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClearPropagationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClearPropagationDUnitTest.java
@@ -18,7 +18,7 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionEvent;
 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.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.DistributedSystem;
@@ -402,7 +402,7 @@ public class ClearPropagationDUnitTest extends DistributedTestCase
     factory.setDataPolicy(DataPolicy.REPLICATE);
     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/tier/sockets/ClientConflationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientConflationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientConflationDUnitTest.java
index cf95b24..90ec2b5 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientConflationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientConflationDUnitTest.java
@@ -22,15 +22,15 @@ 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.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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
@@ -123,7 +123,7 @@ public class ClientConflationDUnitTest extends DistributedTestCase
     createClientCacheFeeder(getServerHostName(Host.getHost(0)), new Integer(PORT));
     vm1.invoke(ClientConflationDUnitTest.class, "createClientCache", new Object[] { getServerHostName(vm1.getHost()), new Integer(PORT),
       conflation});
-    vm1.invoke(ClientConflationDUnitTest.class, "setBridgeObserverForBeforeInterestRecovery");
+    vm1.invoke(ClientConflationDUnitTest.class, "setClientServerObserverForBeforeInterestRecovery");
     vm1.invoke(ClientConflationDUnitTest.class, "setAllCountersZero");
     vm1.invoke(ClientConflationDUnitTest.class, "assertAllCountersZero");
     vm1.invoke(ClientConflationDUnitTest.class, "registerInterest");
@@ -267,10 +267,10 @@ public class ClientConflationDUnitTest extends DistributedTestCase
    * reset all counters to zero before interest recovery
    *
    */
-  public static void setBridgeObserverForBeforeInterestRecovery()
+  public static void setClientServerObserverForBeforeInterestRecovery()
   {
     PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       public void beforeInterestRecovery()
       {
         setAllCountersZero();
@@ -282,9 +282,9 @@ public class ClientConflationDUnitTest extends DistributedTestCase
    * Assert all queues are empty to aid later assertion for listener event counts.
    */
   public static void assertAllQueuesEmpty() {
-    Iterator servers = cacheServer.getBridgeServers().iterator();
+    Iterator servers = cacheServer.getCacheServers().iterator();
     while (servers.hasNext()) {
-      Iterator proxies = ((BridgeServerImpl)servers.next()).getAcceptor().
+      Iterator proxies = ((CacheServerImpl)servers.next()).getAcceptor().
         getCacheClientNotifier().getClientProxies().iterator();
       while (proxies.hasNext()) {
         int qsize = ((CacheClientProxy)proxies.next()).getQueueSize();
@@ -402,7 +402,7 @@ public class ClientConflationDUnitTest extends DistributedTestCase
     RegionAttributes attrs2 = factory.create();
     cacheServer.createRegion(REGION_NAME1, attrs1);
     cacheServer.createRegion(REGION_NAME2, attrs2);
-    BridgeServer server = cacheServer.addBridgeServer();
+    CacheServer server = cacheServer.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
     server.setPort(port);
     server.setNotifyBySubscription(true);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java
index 0891524..015bb59 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitorJUnitTest.java
@@ -28,7 +28,7 @@ import com.gemstone.gemfire.cache.client.ServerConnectivityException;
 import com.gemstone.gemfire.cache.client.internal.Connection;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+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.EventID;
@@ -115,7 +115,7 @@ public class ClientHealthMonitorJUnitTest
    */
   private int createServer()
   {
-    BridgeServer server = null;
+    CacheServer server = null;
     try {
       Properties p = new Properties();
       // make it a loner
@@ -124,7 +124,7 @@ public class ClientHealthMonitorJUnitTest
       
       this.system = DistributedSystem.connect(p);
       this.cache = CacheFactory.create(system);
-      server = this.cache.addBridgeServer();
+      server = this.cache.addCacheServer();
       int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
       server.setMaximumTimeBetweenPings(TIME_BETWEEN_PINGS);
       server.setMaxThreads(getMaxThreads());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientInterestNotifyDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientInterestNotifyDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientInterestNotifyDUnitTest.java
index 551bebd..c66291e 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientInterestNotifyDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientInterestNotifyDUnitTest.java
@@ -23,12 +23,12 @@ import com.gemstone.gemfire.cache.InterestResultPolicy;
 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.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;
 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 dunit.DistributedTestCase;
@@ -402,10 +402,10 @@ public class ClientInterestNotifyDUnitTest extends DistributedTestCase
   // occasional failures in precheckins and cruisecontrol.
   
   public static void assertAllQueuesEmpty() {
-    Iterator servers = cacheServer.getBridgeServers().iterator();
+    Iterator servers = cacheServer.getCacheServers().iterator();
     assertTrue("No servers found!", servers.hasNext());
     while (servers.hasNext()) {
-      Iterator proxies = ((BridgeServerImpl)servers.next()).getAcceptor().
+      Iterator proxies = ((CacheServerImpl)servers.next()).getAcceptor().
         getCacheClientNotifier().getClientProxies().iterator();
       assertTrue("No proxies found!", proxies.hasNext());
       while (proxies.hasNext()) {
@@ -420,8 +420,8 @@ public class ClientInterestNotifyDUnitTest extends DistributedTestCase
       String excuse;
       public boolean done() {
         // assume a single cache server as configured in this test
-        BridgeServerImpl bridgeServer = (BridgeServerImpl) cacheServer.
-          getBridgeServers().iterator().next();
+        CacheServerImpl bridgeServer = (CacheServerImpl) cacheServer.
+          getCacheServers().iterator().next();
         if (bridgeServer == null) {
           excuse = "No Cache Server";
           return false;
@@ -474,7 +474,7 @@ public class ClientInterestNotifyDUnitTest extends DistributedTestCase
     cacheServer.createRegion(REGION_NAME1, attrs);
     cacheServer.createRegion(REGION_NAME2, attrs);
     cacheServer.createRegion(REGION_NAME3, attrs);
-    BridgeServer server = cacheServer.addBridgeServer();
+    CacheServer server = cacheServer.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
     server.setPort(port);
     server.setNotifyBySubscription(true);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscDUnitTest.java
new file mode 100755
index 0000000..1813123
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscDUnitTest.java
@@ -0,0 +1,1381 @@
+/*=========================================================================
+ * 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.tier.sockets;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.CacheWriterException;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.client.NoAvailableServersException;
+import com.gemstone.gemfire.cache.client.Pool;
+import com.gemstone.gemfire.cache.client.PoolManager;
+import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.client.internal.RegisterInterestTracker;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
+import com.gemstone.gemfire.cache30.CacheTestCase;
+import com.gemstone.gemfire.cache30.LRUEvictionControllerDUnitTest;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.SerializableRunnable;
+import dunit.VM;
+
+import java.util.Iterator;
+import java.util.Properties;
+import java.util.Set;
+
+import junit.framework.AssertionFailedError;
+
+/**
+ * Tests client server corner cases between Region and Pool
+ *
+ * @author Yogesh Mahajan
+ *
+ */
+public class ClientServerMiscDUnitTest extends CacheTestCase
+{
+//  private static Cache cache = null;
+
+  protected static PoolImpl pool = null;
+
+  protected static Connection conn = null;
+  
+  private static Cache static_cache;
+
+  private static int PORT1;
+
+  private static final String k1 = "k1";
+
+  private static final String k2 = "k2";
+
+  private static final String server_k1 = "server-k1";
+
+  private static final String server_k2 = "server-k2";
+
+  private static final String REGION_NAME1 = "ClientServerMiscDUnitTest_region1";
+
+  private static final String REGION_NAME2 = "ClientServerMiscDUnitTest_region2";
+
+  private static final String PR_REGION_NAME = "ClientServerMiscDUnitTest_PRregion";
+
+  private static Host host;
+
+  private static VM server1;
+
+  private static VM server2;
+
+  private static RegionAttributes attrs;
+
+
+  // variables for concurrent map API test
+  Properties props = new Properties();
+  final int putRange_1Start = 1;
+  final int putRange_1End = 5;
+  final int putRange_2Start = 6;
+  final int putRange_2End = 10;
+  final int putRange_3Start = 11;
+  final int putRange_3End = 15;
+  final int putRange_4Start = 16;
+  final int putRange_4End = 20;
+  final int removeRange_1Start = 2;
+  final int removeRange_1End = 4;
+  final int removeRange_2Start = 7;
+  final int removeRange_2End = 9;
+
+  
+  
+  /** constructor */
+  public ClientServerMiscDUnitTest(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception
+  {
+    super.setUp();
+    host = Host.getHost(0);
+    server1 = host.getVM(0);
+    server2 = host.getVM(1);
+  }
+
+  private int initServerCache(boolean notifyBySub) {
+    Object[] args = new Object[] { notifyBySub, getMaxThreads()};
+    return ((Integer)server1.invoke(ClientServerMiscDUnitTest.class,
+                                    "createServerCache",
+                                    args)).intValue();
+  }
+  
+  private int initServerCache2(boolean notifyBySub) {
+    Object[] args = new Object[] {notifyBySub, getMaxThreads()};
+    return ((Integer)server2.invoke(ClientServerMiscDUnitTest.class,
+                                    "createServerCache",
+                                    args)).intValue();
+  }
+
+  
+  public void testConcurrentOperationsWithDRandPR() throws Exception {
+    int port1 = initServerCache(true); // vm0
+    int port2 = initServerCache2(true); // vm1
+    String serverName = getServerHostName(Host.getHost(0));
+    host.getVM(2).invoke(this.getClass(), "createClientCacheV", new Object[]{serverName, port1});
+    host.getVM(3).invoke(this.getClass(), "createClientCacheV", new Object[]{serverName, port2});
+    getLogWriter().info("Testing concurrent map operations from a client with a distributed region");
+    concurrentMapTest(host.getVM(2), "/" + REGION_NAME1);
+    // TODO add verification in vm3
+    getLogWriter().info("Testing concurrent map operations from a client with a partitioned region");
+    concurrentMapTest(host.getVM(2), "/" + PR_REGION_NAME);
+    // TODO add verification in vm3
+  }
+
+  public void testConcurrentOperationsWithDRandPRandEmptyClient() throws Exception {
+    int port1 = initServerCache(true); // vm0
+    int port2 = initServerCache2(true); // vm1
+    String serverName = getServerHostName(Host.getHost(0));
+    host.getVM(2).invoke(this.getClass(), "createEmptyClientCache", new Object[]{serverName, port1});
+    host.getVM(3).invoke(this.getClass(), "createClientCacheV", new Object[]{serverName, port2});
+    getLogWriter().info("Testing concurrent map operations from a client with a distributed region");
+    concurrentMapTest(host.getVM(2), "/" + REGION_NAME1);
+    // TODO add verification in vm3
+    getLogWriter().info("Testing concurrent map operations from a client with a partitioned region");
+    concurrentMapTest(host.getVM(2), "/" + PR_REGION_NAME);
+    // TODO add verification in vm3
+  }
+
+  /**
+   * Do putIfAbsent(), replace(Object, Object),
+   * replace(Object, Object, Object), remove(Object, Object) operations
+   */
+  public void concurrentMapTest(final VM clientVM, final String rName) {
+    
+    //String exceptionStr = "";
+    clientVM.invoke(new CacheSerializableRunnable("doConcurrentMapOperations") {
+      public void run2() throws CacheException {
+        Cache cache = getCache();
+        final Region pr = cache.getRegion(rName);
+        assertNotNull(rName + " not created", pr);
+        boolean isEmpty = pr.getAttributes().getDataPolicy() == DataPolicy.EMPTY;
+        
+        // test successful putIfAbsent
+        for (int i = putRange_1Start; i <= putRange_1End; i++) {
+          Object putResult = pr.putIfAbsent(Integer.toString(i),
+                                            Integer.toString(i));
+          assertNull("Expected null, but got " + putResult + " for key " + i,
+                     putResult);
+        }
+        int size;
+        if (!isEmpty) {
+          size = pr.size();
+          assertEquals("Size doesn't return expected value", putRange_1End, size);
+          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
+              pr.isEmpty());
+        }
+        
+        // test unsuccessful putIfAbsent
+        for (int i = putRange_1Start; i <= putRange_1End; i++) {
+          Object putResult = pr.putIfAbsent(Integer.toString(i),
+                                            Integer.toString(i + 1));
+          assertEquals("for i=" + i, Integer.toString(i), putResult);
+          assertEquals("for i=" + i, Integer.toString(i), pr.get(Integer.toString(i)));
+        }
+        if (!isEmpty) {
+          size = pr.size();
+          assertEquals("Size doesn't return expected value", putRange_1End, size);
+          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
+              pr.isEmpty());
+        }
+               
+        // test successful replace(key, oldValue, newValue)
+        for (int i = putRange_1Start; i <= putRange_1End; i++) {
+         boolean replaceSucceeded = pr.replace(Integer.toString(i),
+                                               Integer.toString(i),
+                                               "replaced" + i);
+          assertTrue("for i=" + i, replaceSucceeded);
+          assertEquals("for i=" + i, "replaced" + i, pr.get(Integer.toString(i)));
+        }
+        if (!isEmpty) {
+          size = pr.size();
+          assertEquals("Size doesn't return expected value", putRange_1End, size);
+          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
+              pr.isEmpty());
+        }
+        
+        // test unsuccessful replace(key, oldValue, newValue)
+        for (int i = putRange_1Start; i <= putRange_2End; i++) {
+         boolean replaceSucceeded = pr.replace(Integer.toString(i),
+                                               Integer.toString(i), // wrong expected old value
+                                               "not" + i);
+         assertFalse("for i=" + i, replaceSucceeded);
+         assertEquals("for i=" + i,
+                      i <= putRange_1End ? "replaced" + i : null,
+                      pr.get(Integer.toString(i)));
+        }
+        if (!isEmpty) {
+          size = pr.size();
+          assertEquals("Size doesn't return expected value", putRange_1End, size);
+          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
+              pr.isEmpty());
+        }
+                                    
+        // test successful replace(key, value)
+        for (int i = putRange_1Start; i <= putRange_1End; i++) {
+          Object replaceResult = pr.replace(Integer.toString(i),
+                                            "twice replaced" + i);
+          assertEquals("for i=" + i, "replaced" + i, replaceResult);
+          assertEquals("for i=" + i,
+                       "twice replaced" + i,
+                       pr.get(Integer.toString(i)));
+        }
+        if (!isEmpty) {
+          size = pr.size();
+          assertEquals("Size doesn't return expected value", putRange_1End, size);
+          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
+              pr.isEmpty());
+        }
+                                    
+        // test unsuccessful replace(key, value)
+        for (int i = putRange_2Start; i <= putRange_2End; i++) {
+          Object replaceResult = pr.replace(Integer.toString(i),
+                                           "thrice replaced" + i);
+          assertNull("for i=" + i, replaceResult);
+          assertNull("for i=" + i, pr.get(Integer.toString(i)));
+        }
+        if (!isEmpty) {
+          size = pr.size();
+          assertEquals("Size doesn't return expected value", putRange_1End, size);
+          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
+              pr.isEmpty());
+        }
+                                    
+        // test unsuccessful remove(key, value)
+        for (int i = putRange_1Start; i <= putRange_2End; i++) {
+          boolean removeResult = pr.remove(Integer.toString(i),
+                                           Integer.toString(-i));
+          assertFalse("for i=" + i, removeResult);
+          assertEquals("for i=" + i,
+                       i <= putRange_1End ? "twice replaced" + i : null,
+                       pr.get(Integer.toString(i)));
+        }
+        if (!isEmpty) {
+          size = pr.size();
+          assertEquals("Size doesn't return expected value", putRange_1End, size);
+          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
+              pr.isEmpty());
+        }
+
+        // test successful remove(key, value)
+        for (int i = putRange_1Start; i <= putRange_1End; i++) {
+          boolean removeResult = pr.remove(Integer.toString(i),
+                                           "twice replaced" + i);
+          assertTrue("for i=" + i, removeResult);
+          assertEquals("for i=" + i, null, pr.get(Integer.toString(i)));
+        }
+        if (!isEmpty) {
+          size = pr.size();
+          assertEquals("Size doesn't return expected value", 0, size);
+          pr.localClear();
+          assertTrue("isEmpty doesnt return proper state of the PartitionedRegion", 
+              pr.isEmpty());
+        }
+        
+        if (!isEmpty) {
+          // bug #42169 - entry not updated on server when locally destroyed on client
+          String key42169 = "key42169";
+          pr.put(key42169, "initialValue42169");
+          pr.localDestroy(key42169);
+          boolean success = pr.replace(key42169, "initialValue42169", "newValue42169");
+          assertTrue("expected replace to succeed", success);
+          pr.destroy(key42169);
+          pr.put(key42169, "secondRound");
+          pr.localDestroy(key42169);
+          Object result = pr.putIfAbsent(key42169, null);
+          assertEquals("expected putIfAbsent to fail", result, "secondRound");
+          pr.destroy(key42169);
+        }
+        
+        if (isEmpty) {
+          String key41265 = "key41265";
+          boolean success = pr.remove(key41265, null);
+          assertFalse("expected remove to fail because key does not exist", success);
+        }
+        
+        // test null values
+        
+        // putIfAbsent with null value creates invalid entry
+        Object oldValue = pr.putIfAbsent("keyForNull", null);
+        assertNull(oldValue);
+        if (!isEmpty) {
+          assertTrue(pr.containsKey("keyForNull"));
+          assertTrue(!pr.containsValueForKey("keyForNull"));
+        }
+        
+        // replace allows null value for oldValue, meaning invalidated entry
+        assertTrue(pr.replace("keyForNull", null, "no longer invalid"));
+        
+        // replace does not allow null value for new value
+        try {
+          pr.replace("keyForNull", "no longer invalid", null);
+          fail("expected a NullPointerException");
+        } catch (NullPointerException expected) {
+        }
+        
+        // other variant of replace does not allow null value for new value
+        try {
+          pr.replace("keyForNull", null);
+          fail ("expected a NullPointerException");
+        } catch (NullPointerException expected) {
+        }
+        
+        // replace with null oldvalue matches invalidated entry
+        pr.putIfAbsent("otherKeyForNull", null);
+        int puts = ((GemFireCacheImpl)pr.getCache()).getCachePerfStats().getPuts();
+        boolean success = pr.replace("otherKeyForNull", null, "no longer invalid");
+        assertTrue(success);
+        int newputs = ((GemFireCacheImpl)pr.getCache()).getCachePerfStats().getPuts();
+        assertTrue("stats not updated properly or replace malfunctioned", newputs == puts+1);
+
+      }
+    });
+  }
+
+  /**
+   * Test two regions: notify by subscription is true.
+   * For region1 the interest list is empty , for region 2 the intetest list is all keys.
+   * If an update/create is made on region1 , the client should not receive any.
+   * If the create/update is on region2 , the client should receive the update.
+   */
+  public void testForTwoRegionHavingDifferentInterestList()
+      throws Exception
+  {
+    // start server first
+    PORT1 = initServerCache(true);
+    createClientCache(getServerHostName(Host.getHost(0)), PORT1);
+    populateCache();
+    registerInterest();
+    server1.invoke(ClientServerMiscDUnitTest.class, "put");
+
+//    pause(5000 + 5000 + 10000);
+    /*final int maxWaitTime = Integer.getInteger(WAIT_PROPERTY, WAIT_DEFAULT).intValue();
+    try {
+      Thread.yield();
+       Thread.sleep(maxWaitTime);
+    }
+    catch (InterruptedException e) {
+      fail("interrupted");
+    }*/
+    verifyUpdates();
+
+  }
+
+  /**
+   * Test two regions: notify by subscription is true.
+   * Both the regions have registered interest in all the keys.
+   * Now close region1 on the client.
+   * The region1 should get removed from the interest list on CCP at server.
+   * Any update on region1 on server should not get pushed to the client.
+   * Ensure that the message related is not added to the client's queue at all
+   * ( which is diferent from not receiving a callbak on the client).
+   * If an update on region2 is made on the server , then client should receive the calback
+   */
+  public void testForTwoRegionHavingALLKEYSInterest()
+      throws Exception
+  {
+    // start server first
+    PORT1 = initServerCache(true);
+    createClientCache(getServerHostName(Host.getHost(0)), PORT1);
+    populateCache();
+    registerInterestInBothTheRegions();
+    closeRegion1();
+    pause(6000);
+    server1.invoke(ClientServerMiscDUnitTest.class,
+        "verifyInterestListOnServer");
+    server1.invoke(ClientServerMiscDUnitTest.class, "put");
+    //pause(5000);
+    verifyUpdatesOnRegion2();
+  }
+
+  /** Test two regions: notify by subscription is true.
+   * Both the regions have registered interest in all the keys.
+   * Close both the regions. When the last region is closed ,
+   * it should close the ConnectionProxy on the client ,
+   * close all the server connection threads on the server &
+   * remove the CacheClientProxy from the CacheClient notifier
+   */
+  public void testRegionClose() throws Exception
+  {
+    // start server first
+    PORT1 = initServerCache(true);
+    pool = (PoolImpl)createClientCache(getServerHostName(Host.getHost(0)),PORT1);
+    populateCache();
+    registerInterestInBothTheRegions();
+    closeBothRegions();
+    //pause(5000);
+    assertEquals(false, pool.isDestroyed());
+    pool.destroy();
+    assertEquals(true, pool.isDestroyed());
+    server1.invoke(ClientServerMiscDUnitTest.class,
+        "verifyNoCacheClientProxyOnServer");
+
+  }
+
+  /**
+   * Test two regions: notify by
+   * subscription is true. Both the regions have registered interest in all the
+   * keys. Destroy region1 on the client. It should reach the server , kill the
+   * region on the server , propagate it to the interested clients , but it
+   * should keep CacheClient Proxy alive. Destroy Region2 . It should reach
+   * server , close conenction proxy , destroy the region2 on the server ,
+   * remove the cache client proxy from the cache client notifier & propagate it
+   * to the clients. Then create third region and verify that no
+   * CacheClientProxy is created on server
+   */
+  public void testCCPDestroyOnLastDestroyRegion() throws Exception
+  {
+    PORT1 = initServerCache(true);
+    PoolImpl pool = (PoolImpl)createClientCache(getServerHostName(Host.getHost(0)),PORT1);
+    destroyRegion1();
+    // pause(5000);
+    server1.invoke(ClientServerMiscDUnitTest.class,
+        "verifyCacheClientProxyOnServer", new Object[] { new String(
+            REGION_NAME1) });
+    Connection conn = pool.acquireConnection();
+    assertNotNull(conn);
+    assertEquals(1, pool.getConnectedServerCount());
+    assertEquals(false, pool.isDestroyed());
+    destroyRegion2();
+    assertEquals(false, pool.isDestroyed());
+    destroyPRRegion();
+    assertEquals(false, pool.isDestroyed());
+    pool.destroy();
+    assertEquals(true, pool.isDestroyed());
+    // pause(5000);
+    server1.invoke(ClientServerMiscDUnitTest.class,
+        "verifyNoCacheClientProxyOnServer");
+    try {
+      getCache().createRegion(REGION_NAME2, attrs);
+      fail("expected IllegalStateException");
+    } catch (IllegalStateException expected) {
+    }
+  }
+
+  /**
+   * Test two regions:If notify by
+   * subscription is false , both the regions should receive invalidates for the
+   * updates on server in their respective regions
+   *
+   */
+  public void testInvalidatesPropagateOnTwoRegions()
+      throws Exception
+  {
+    // start server first
+    PORT1 = initServerCache(false);
+    createClientCache(getServerHostName(Host.getHost(0)), PORT1);
+    registerInterestForInvalidatesInBothTheRegions();
+    populateCache();
+    server1.invoke(ClientServerMiscDUnitTest.class, "put");
+    //pause(5000);
+    verifyInvalidatesOnBothRegions();
+
+  }
+
+  /**
+   * Test for bug 43407, where LRU in the client caused an entry to be
+   * evicted with DESTROY(), then the client invalidated the entry and
+   * did a get().  After the get() the entry was not seen to be in the
+   * client's cache.  This turned out to be expected behavior, but we
+   * now have this test to guarantee that the product behaves as expected.
+   */
+  public void testBug43407()
+      throws Exception
+  {
+    // start server first
+    PORT1 = initServerCache(false);
+    createClientCache(getServerHostName(Host.getHost(0)), PORT1);
+    registerInterestForInvalidatesInBothTheRegions();
+    Region region = static_cache.getRegion(REGION_NAME1);
+    populateCache();
+    region.put("invalidationKey", "invalidationValue");
+    region.localDestroy("invalidationKey");
+    if (region.containsKey("invalidationKey")) {
+      fail("region still contains invalidationKey");
+    }
+    region.invalidate("invalidationKey");
+    if (region.containsKey("invalidationKey")) {
+      fail("this test expects the entry is not created on invalidate() if not there before the operation");
+    }
+    Object value = region.get("invalidationKey");
+    if (value != null) {
+      fail("this test expected a null response to get('invalidationKey')");
+    }
+    if (!region.containsKeyOnServer("invalidationKey")) {
+      fail("expected an entry on the server after invalidation");
+    }
+    // bug 43407 asserts that there should be an entry, but the product does not
+    // do this.  This verifies that the product does not behave as asserted in that bug
+    if (region.containsKey("invalidationKey")) {
+      fail("expected no entry after invalidation when entry was not in client but was on server");
+    }
+  }
+
+  /**
+   * Create cache, create pool, notify-by-subscription=false,
+   * create a region and on client and on server.
+   * Do not attach pool to region ,
+   * populate some entries on region both on client and server.
+   * Update the entries on server the client.
+   * The client should not have entry invalidate.
+   *
+   * @throws Exception
+   */
+  public void testInvalidatesPropagateOnRegionHavingNoPool()
+      throws Exception
+  {
+    // start server first
+    PORT1 = initServerCache(false);
+    Properties props = new Properties();
+    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    new ClientServerMiscDUnitTest("temp").createCache(props);
+    String host = getServerHostName(server1.getHost());
+    PoolImpl p = (PoolImpl)PoolManager.createFactory()
+      .addServer(host, PORT1)
+      .setSubscriptionEnabled(true)
+      .setThreadLocalConnections(true)
+      .setReadTimeout(1000)
+      .setSocketBufferSize(32768)
+      .setMinConnections(3)
+      .setSubscriptionRedundancy(-1)
+      .setPingInterval(2000)
+      // .setRetryAttempts(5)
+      // .setRetryInterval(2000)
+      .create("testInvalidatesPropagateOnRegionHavingNoPool");
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    // factory.setPoolName(p.getName());
+
+    attrs = factory.create();
+    final Region region1 = getCache().createRegion(REGION_NAME1, attrs);
+    final Region region2 = getCache().createRegion(REGION_NAME2, attrs);
+    assertNotNull(region1);
+    assertNotNull(region2);
+    pool = p;
+    conn = pool.acquireConnection();
+    assertNotNull(conn);
+
+    populateCache();
+    server1.invoke(ClientServerMiscDUnitTest.class, "put");
+    // pause(5000);
+    WaitCriterion wc = new WaitCriterion() {
+      String excuse;
+      public boolean done() {
+        Object val = region1.getEntry(k1).getValue();
+        return k1.equals(val);
+      }
+      public String description() {
+        return excuse;
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+    
+    // assertEquals(region1.getEntry(k1).getValue(), k1);
+    wc = new WaitCriterion() {
+      String excuse;
+      public boolean done() {
+        Object val = region1.getEntry(k2).getValue();
+        return k2.equals(val);
+      }
+      public String description() {
+        return excuse;
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+    
+    wc = new WaitCriterion() {
+      String excuse;
+      public boolean done() {
+        Object val = region2.getEntry(k1).getValue();
+        return k1.equals(val);
+      }
+      public String description() {
+        return excuse;
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+    
+    // assertEquals(region1.getEntry(k2).getValue(), k2);
+    // assertEquals(region2.getEntry(k1).getValue(), k1);
+    wc = new WaitCriterion() {
+      String excuse;
+      public boolean done() {
+        Object val = region2.getEntry(k2).getValue();
+        return k2.equals(val);
+      }
+      public String description() {
+        return excuse;
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+    
+    // assertEquals(region2.getEntry(k2).getValue(), k2);
+  }
+
+  /**
+   * Create proxy before cache creation, create cache, create two regions,
+   * attach same bridge writer to both of the regions Region interests AL_KEYS
+   * on both the regions, notify-by-subscription=true . The CCP should have both
+   * the regions in interest list.
+   *
+   * @throws Exception
+   */
+
+  public void testProxyCreationBeforeCacheCreation() throws Exception
+  {
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    DistributedSystem ds = getSystem(props);
+    assertNotNull(ds);
+    ds.disconnect();
+    ds = getSystem(props);
+    PORT1 = initServerCache(true);
+    String host = getServerHostName(server1.getHost());
+    Pool p = PoolManager.createFactory()
+      .addServer(host, PORT1)
+      .setSubscriptionEnabled(true)
+      .setSubscriptionRedundancy(-1)
+      // .setRetryAttempts(5)
+      .create("testProxyCreationBeforeCacheCreationPool");
+
+    Cache cache = getCache();
+    assertNotNull(cache);
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setPoolName(p.getName());
+
+    RegionAttributes myAttrs = factory.create();
+    Region region1 = cache.createRegion(REGION_NAME1, myAttrs);
+    Region region2 = cache.createRegion(REGION_NAME2, myAttrs);
+    assertNotNull(region1);
+    assertNotNull(region2);
+    //region1.registerInterest(CacheClientProxy.ALL_KEYS);
+    region2.registerInterest("ALL_KEYS");
+    pause(6000);
+    server1.invoke(ClientServerMiscDUnitTest.class,
+        "verifyInterestListOnServer");
+
+  }
+  /**
+   * 
+   * Cycling a DistributedSystem with an initialized pool causes interest registration NPE
+   * 
+   * Test Scenario:
+   *  
+   * Create a DistributedSystem (DS1). 
+   * Create a pool, initialize (creates a proxy with DS1 memberid) 
+   * Disconnect DS1.  Create a DistributedSystem (DS2). 
+   * Create a Region with pool, it attempts to register interest using DS2 memberid, gets NPE.
+   *  
+   * @throws Exception
+   */
+  public void testBug35380() throws Exception
+  {
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    DistributedSystem ds = getSystem(props);
+    assertNotNull(ds);
+    
+    PORT1 = initServerCache(true);
+    String host = getServerHostName(server1.getHost());
+    Pool p = PoolManager.createFactory()
+      .addServer(host, PORT1)
+      .setSubscriptionEnabled(true)
+      .setSubscriptionRedundancy(-1)
+      //.setRetryAttempts(5)
+      .create("testBug35380Pool");
+
+    Cache cache = getCache();
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setPoolName(p.getName());
+
+    RegionAttributes myAttrs = factory.create();
+    Region region1 = cache.createRegion(REGION_NAME1, myAttrs);
+    Region region2 = cache.createRegion(REGION_NAME2, myAttrs);
+    assertNotNull(region1);
+    assertNotNull(region2);
+    
+    region2.registerInterest("ALL_KEYS");
+    
+    ds.disconnect();
+    Properties prop = new Properties();
+    prop.setProperty("mcast-port", "0");
+    prop.setProperty("locators", "");
+    ds = getSystem(prop);
+    
+    cache = getCache();
+    assertNotNull(cache);
+
+    AttributesFactory factory1 = new AttributesFactory();
+    factory1.setScope(Scope.DISTRIBUTED_ACK);
+    //reuse writer from prev DS
+    factory1.setPoolName(p.getName());
+
+    RegionAttributes attrs1 = factory1.create();
+    try {
+      cache.createRegion(REGION_NAME1, attrs1);
+      fail("expected ShutdownException");
+    }
+    catch(IllegalStateException expected) {
+    }
+    catch (DistributedSystemDisconnectedException expected) {
+    } 
+  }
+  
+
+  private void createCache(Properties props) throws Exception {
+    createCacheV(props);
+  }
+  private Cache createCacheV(Properties props) throws Exception
+  {
+    DistributedSystem ds = getSystem(props);
+    assertNotNull(ds);
+    ds.disconnect();
+    ds = getSystem(props);
+    Cache cache = getCache();
+    assertNotNull(cache);
+    return cache;
+  }
+
+  public static void createClientCacheV(String h, int port)
+  throws Exception {
+    _createClientCache(h, port, false);
+  }
+
+  public static void createEmptyClientCache(String h, int port)
+  throws Exception {
+    _createClientCache(h, port, true);
+  }
+
+  public static Pool createClientCache(String h, int port)
+  throws Exception  {
+    return _createClientCache(h, port, false);
+  }
+  
+  public static Pool _createClientCache(String h, int port, boolean empty)
+  throws Exception  {
+    Properties props = new Properties();
+    props.setProperty("mcast-port", "0");
+    props.setProperty("locators", "");
+    Cache cache = new ClientServerMiscDUnitTest("temp").createCacheV(props);
+    ClientServerMiscDUnitTest.static_cache = cache;
+    PoolImpl p = (PoolImpl)PoolManager.createFactory()
+      .addServer(h, port)
+      .setSubscriptionEnabled(true)
+      .setThreadLocalConnections(true)
+      .setReadTimeout(1000)
+      .setSocketBufferSize(32768)
+      .setMinConnections(3)
+      .setSubscriptionRedundancy(-1)
+      .setPingInterval(2000)
+      // .setRetryAttempts(5)
+      // .setRetryInterval(2000)
+      .create("ClientServerMiscDUnitTestPool");
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    if (empty) {
+      factory.setDataPolicy(DataPolicy.EMPTY);
+    }
+    factory.setPoolName(p.getName());
+
+    attrs = factory.create();
+    Region region1 = cache.createRegion(REGION_NAME1, attrs);
+    Region region2 = cache.createRegion(REGION_NAME2, attrs);
+    Region prRegion = cache.createRegion(PR_REGION_NAME, attrs);
+    assertNotNull(region1);
+    assertNotNull(region2);
+    assertNotNull(prRegion);
+    pool = p;
+//    conn = pool.acquireConnection();
+//    assertNotNull(conn);
+    // TODO does this WaitCriterion actually help?
+    WaitCriterion wc = new WaitCriterion() {
+      String excuse;
+      public boolean done() {
+        try {
+          conn = pool.acquireConnection();
+          if (conn == null) {
+            excuse = "acquireConnection returned null?";
+            return false;
+          }
+          return true;
+        } catch (NoAvailableServersException e) {
+          excuse = "Cannot find a server: " + e;
+          return false;
+        }
+      }
+      public String description() {
+        return excuse;
+      }
+    };
+    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+    
+    return p;
+  }
+
+  public static Integer createServerCache(Boolean notifyBySubscription, Integer maxThreads)
+  throws Exception {
+    Cache cache = new ClientServerMiscDUnitTest("temp").createCacheV(new Properties());
+    unsetSlowDispatcherFlag();
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setEnableConflation(true);
+    factory.setDataPolicy(DataPolicy.REPLICATE);
+    RegionAttributes myAttrs = factory.create();
+    Region r1 = cache.createRegion(REGION_NAME1, myAttrs);
+    Region r2 = cache.createRegion(REGION_NAME2, myAttrs);
+    factory = new AttributesFactory();
+    factory.setDataPolicy(DataPolicy.PARTITION);
+    RegionAttributes prAttrs = factory.create();
+    Region pr = cache.createRegion(PR_REGION_NAME, prAttrs);
+    assertNotNull(r1);
+    assertNotNull(r2);
+    assertNotNull(pr);
+
+    CacheServer server = cache.addCacheServer();
+    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+    r1.getCache().getDistributedSystem().getLogWriter().info("Starting server on port " + port);
+    server.setPort(port);
+    server.setMaxThreads(maxThreads.intValue());
+    server.setNotifyBySubscription(notifyBySubscription.booleanValue());
+    server.start();
+    r1.getCache().getDistributedSystem().getLogWriter().info("Started server on port " + server.getPort());
+    return new Integer(server.getPort());
+
+  }
+
+  protected int getMaxThreads() {
+    return 0; 
+  }
+
+  public static void registerInterest()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      assertNotNull(r);
+      //r.registerInterestRegex(CacheClientProxy.ALL_KEYS);
+      r.registerInterest("ALL_KEYS");
+    }
+    catch (CacheWriterException e) {
+      e.printStackTrace();
+      fail("Test failed due to CacheWriterException during registerInterest", e);
+    }
+  }
+
+  public static void registerInterestForInvalidatesInBothTheRegions()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
+      assertNotNull(r1);
+      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      assertNotNull(r2);
+      r1.registerInterest("ALL_KEYS", false, false);
+      r2.registerInterest("ALL_KEYS", false, false);
+    }
+    catch (CacheWriterException e) {
+      e.printStackTrace();
+      fail(
+          "Test failed due to CacheWriterException during registerInterestnBothRegions",
+          e);
+    }
+  }
+
+  public static void registerInterestInBothTheRegions()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
+      assertNotNull(r1);
+      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      assertNotNull(r2);
+      r1.registerInterest("ALL_KEYS");
+      r2.registerInterest("ALL_KEYS");
+    }
+    catch (CacheWriterException e) {
+      e.printStackTrace();
+      fail(
+          "Test failed due to CacheWriterException during registerInterestnBothRegions",
+          e);
+    }
+  }
+
+  public static void closeRegion1()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
+      assertNotNull(r1);
+      r1.close();
+    }
+    catch (Exception e) {
+      e.printStackTrace();
+      fail("Test failed due to Exception during closeRegion1", e);
+    }
+  }
+
+  public static void closeBothRegions()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
+      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      Region pr = cache.getRegion(Region.SEPARATOR + PR_REGION_NAME);
+      assertNotNull(r1);
+      assertNotNull(r2);
+      assertNotNull(pr);
+      r1.close();
+      r2.close();
+      pr.close();
+    }
+    catch (Exception e) {
+      e.printStackTrace();
+      fail("Test failed due to Exception during closeBothRegions", e);
+    }
+  }
+
+  public static void destroyRegion1()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
+      assertNotNull(r1);
+      r1.destroyRegion();
+    }
+    catch (Exception e) {
+      e.printStackTrace();
+      fail("Test failed due to Exception during closeBothRegions", e);
+    }
+  }
+
+  public static void destroyRegion2()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      assertNotNull(r2);
+      r2.destroyRegion();
+    }
+    catch (Exception e) {
+      e.printStackTrace();
+      fail("Test failed due to Exception during closeBothRegions", e);
+    }
+  }
+
+  public static void destroyPRRegion()  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r2 = cache.getRegion(Region.SEPARATOR + PR_REGION_NAME);
+      assertNotNull(r2);
+      r2.destroyRegion();
+    } catch (Exception e) {
+     // e.printStackTrace();
+      fail("Test failed due to Exception during closeBothRegions", e);
+    }
+  }
+
+  public static void verifyInterestListOnServer()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      assertEquals("More than one BridgeServer", 1, cache.getCacheServers()
+          .size());
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
+          .iterator().next();
+      assertNotNull(bs);
+      assertNotNull(bs.getAcceptor());
+      assertNotNull(bs.getAcceptor().getCacheClientNotifier());
+      Iterator iter_prox = bs.getAcceptor().getCacheClientNotifier()
+          .getClientProxies().iterator();
+      while (iter_prox.hasNext()) {
+        CacheClientProxy ccp = (CacheClientProxy)iter_prox.next();
+        // CCP should not contain region1
+        Set akr = ccp.cils[RegisterInterestTracker.interestListIndex].regions;
+        assertNotNull(akr);
+        assertTrue(!akr.contains(Region.SEPARATOR + REGION_NAME1));
+        // CCP should contain region2
+        assertTrue(akr.contains(Region.SEPARATOR + REGION_NAME2));
+        assertEquals(1, akr.size());
+      }
+    }
+    catch (Exception ex) {
+      ex.printStackTrace();
+      fail("while setting verifyInterestListOnServer  " + ex);
+    }
+  }
+
+  public static void verifyNoCacheClientProxyOnServer()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      assertEquals("More than one BridgeServer", 1, cache.getCacheServers()
+          .size());
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
+          .iterator().next();
+      assertNotNull(bs);
+      assertNotNull(bs.getAcceptor());
+      final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+
+      assertNotNull(ccn);
+      WaitCriterion wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          return ccn.getClientProxies().size() == 0;
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 40 * 1000, 1000, true);
+    }
+    catch (Exception ex) {
+      ex.printStackTrace();
+      fail("while setting verifyNoCacheClientProxyOnServer  " + ex);
+    }
+  }
+
+  public static void verifyCacheClientProxyOnServer(String regionName)
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      assertNull(cache.getRegion(Region.SEPARATOR + regionName));
+       verifyCacheClientProxyOnServer();
+
+      //assertEquals(1, bs.getAcceptor().getCacheClientNotifier().getClientProxies().size());
+    }
+    catch (Exception ex) {
+      ex.printStackTrace();
+      fail("while setting verifyNoCacheClientProxyOnServer  " + ex);
+    }
+  }
+
+  public static void verifyCacheClientProxyOnServer()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      assertEquals("More than one BridgeServer", 1, cache.getCacheServers()
+          .size());
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
+          .iterator().next();
+      assertNotNull(bs);
+      assertNotNull(bs.getAcceptor());
+      final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
+
+      assertNotNull(ccn);
+      WaitCriterion wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          return ccn.getClientProxies().size() == 1;
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 40 * 1000, 1000, true);
+    }
+    catch (Exception ex) {
+      ex.printStackTrace();
+      fail("while setting verifyNoCacheClientProxyOnServer  " + ex);
+    }
+  }
+
+  public static void populateCache()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
+      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      assertNotNull(r1);
+      assertNotNull(r2);
+
+      if (!r1.containsKey(k1))
+        r1.create(k1, k1);
+      if (!r1.containsKey(k2))
+        r1.create(k2, k2);
+      if (!r2.containsKey(k1))
+        r2.create(k1, k1);
+      if (!r2.containsKey(k2))
+        r2.create(k2, k2);
+
+      assertEquals(r1.getEntry(k1).getValue(), k1);
+      assertEquals(r1.getEntry(k2).getValue(), k2);
+      assertEquals(r2.getEntry(k1).getValue(), k1);
+      assertEquals(r2.getEntry(k2).getValue(), k2);
+    }
+    catch (Exception ex) {
+      fail("failed while createEntries()", ex);
+    }
+  }
+
+  public static void put()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
+      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      assertNotNull(r1);
+      assertNotNull(r2);
+
+      r1.put(k1, server_k1);
+      r1.put(k2, server_k2);
+
+      r2.put(k1, server_k1);
+      r2.put(k2, server_k2);
+
+      assertEquals(r1.getEntry(k1).getValue(), server_k1);
+      assertEquals(r1.getEntry(k2).getValue(), server_k2);
+      assertEquals(r2.getEntry(k1).getValue(), server_k1);
+      assertEquals(r2.getEntry(k2).getValue(), server_k2);
+    }
+    catch (Exception ex) {
+      fail("failed while put()", ex);
+    }
+  }
+
+  public static void verifyUpdates()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
+      final Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      assertNotNull(r1);
+      assertNotNull(r2);
+      // verify updates
+      WaitCriterion wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r1.getEntry(k1).getValue();
+          return k1.equals(val);
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+      
+      // assertEquals(k1, r1.getEntry(k1).getValue());
+      wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r1.getEntry(k2).getValue();
+          return k2.equals(val);
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+      
+      // assertEquals(k2, r1.getEntry(k2).getValue());
+      wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r2.getEntry(k1).getValue();
+          return server_k1.equals(val);
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+      
+      // assertEquals(server_k1, r2.getEntry(k1).getValue());
+      wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r2.getEntry(k2).getValue();
+          return server_k2.equals(val);
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+      
+      // assertEquals(server_k2, r2.getEntry(k2).getValue());
+    }
+    catch (Exception ex) {
+      fail("failed while verifyUpdates()", ex);
+    }
+  }
+
+  public static void verifyInvalidatesOnBothRegions()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
+      final Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      assertNotNull(r1);
+      assertNotNull(r2);
+      
+      WaitCriterion wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r1.getEntry(k1).getValue();
+          return val == null;
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
+      
+      // assertNull(r1.getEntry(k1).getValue());
+      wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r1.getEntry(k1).getValue();
+          return val == null;
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
+      
+      // assertNull(r1.getEntry(k2).getValue());
+      wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r1.getEntry(k2).getValue();
+          return val == null;
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
+      
+
+      // assertNull(r2.getEntry(k1).getValue());
+      wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r2.getEntry(k1).getValue();
+          return val == null;
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
+      
+      // assertNull(r2.getEntry(k2).getValue());
+      wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r2.getEntry(k2).getValue();
+          return val == null;
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
+    }
+    catch (Exception ex) {
+      fail("failed while verifyInvalidatesOnBothRegions()", ex);
+    }
+  }
+
+  public static void verifyUpdatesOnRegion2()
+  {
+    try {
+      Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+      final Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
+      assertNotNull(r2);
+      WaitCriterion wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r2.getEntry(k1).getValue();
+          return server_k1.equals(val);
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+      
+      // assertEquals(server_k1, r2.getEntry(k1).getValue());
+      wc = new WaitCriterion() {
+        String excuse;
+        public boolean done() {
+          Object val = r2.getEntry(k2).getValue();
+          return server_k2.equals(val);
+        }
+        public String description() {
+          return excuse;
+        }
+      };
+      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
+      
+      // assertEquals(server_k2, r2.getEntry(k2).getValue());
+    }
+    catch (Exception ex) {
+      fail("failed while verifyUpdatesOnRegion2()", ex);
+    }
+  }
+
+  public void tearDown2() throws Exception
+  {
+    super.tearDown2();
+    // close the clients first
+    closeCache();
+    // then close the servers
+    server1.invoke(ClientServerMiscDUnitTest.class, "closeCache");
+
+  }
+
+  public static void closeCache()
+  {
+    Cache cache = new ClientServerMiscDUnitTest("temp").getCache();
+    if (cache != null && !cache.isClosed()) {
+      cache.close();
+      cache.getDistributedSystem().disconnect();
+    }
+  }
+
+  /**
+   * set the boolean for starting the dispatcher thread a bit later to FALSE.
+   * This is just a precaution in case any test set it to true and did not unset
+   * it on completion.
+   *
+   */
+  public static void unsetSlowDispatcherFlag()
+  {
+    CacheClientProxy.isSlowStartForTesting = false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscSelectorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscSelectorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscSelectorDUnitTest.java
new file mode 100644
index 0000000..9b7c13e
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientServerMiscSelectorDUnitTest.java
@@ -0,0 +1,27 @@
+/*=========================================================================
+ * 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.tier.sockets;
+
+
+/**
+ * Just like parent but enables server thread pool
+ * (ie. selector)
+ *
+ * @author darrel
+ *
+ */
+public class ClientServerMiscSelectorDUnitTest extends ClientServerMiscDUnitTest
+{
+  public ClientServerMiscSelectorDUnitTest(String name) {
+    super(name);
+  }
+
+  protected int getMaxThreads() {
+    return 2; 
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java
index 35eb52d..88ec5d0 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ConflationDUnitTest.java
@@ -19,14 +19,14 @@ 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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.HARegion;
 import com.gemstone.gemfire.internal.cache.ha.HAHelper;
@@ -131,7 +131,7 @@ public class ConflationDUnitTest extends DistributedTestCase
       createClientCache1UniqueWriter ( getServerHostName(Host.getHost(0)), new Integer(PORT));
       vm2.invoke(ConflationDUnitTest.class, "createClientCache2UniqueWriter",
           new Object[] { getServerHostName(Host.getHost(0)), new Integer(PORT)});
-      vm2.invoke(ConflationDUnitTest.class, "setBridgeObserverForBeforeInterestRecovery");
+      vm2.invoke(ConflationDUnitTest.class, "setClientServerObserverForBeforeInterestRecovery");
       vm2.invoke(ConflationDUnitTest.class, "setAllCountersZero");
       vm2.invoke(ConflationDUnitTest.class, "assertAllCountersZero");
       vm2.invoke(ConflationDUnitTest.class, "registerInterest");
@@ -162,7 +162,7 @@ public class ConflationDUnitTest extends DistributedTestCase
       createClientCache1CommonWriter( getServerHostName(Host.getHost(0)), new Integer(PORT));
       vm2.invoke(ConflationDUnitTest.class, "createClientCache2CommonWriter",
           new Object[] { getServerHostName(Host.getHost(0)), new Integer(PORT)});
-      vm2.invoke(ConflationDUnitTest.class, "setBridgeObserverForBeforeInterestRecovery");
+      vm2.invoke(ConflationDUnitTest.class, "setClientServerObserverForBeforeInterestRecovery");
       vm2.invoke(ConflationDUnitTest.class, "setAllCountersZero");
       vm2.invoke(ConflationDUnitTest.class, "assertAllCountersZero");
       vm2.invoke(ConflationDUnitTest.class, "registerInterest");
@@ -195,7 +195,7 @@ public class ConflationDUnitTest extends DistributedTestCase
       vm2.invoke(ConflationDUnitTest.class,
           "createClientCache2CommonWriterTest3", new Object[] {
         getServerHostName(Host.getHost(0)), new Integer(PORT) });
-      vm2.invoke(ConflationDUnitTest.class, "setBridgeObserverForBeforeInterestRecovery");
+      vm2.invoke(ConflationDUnitTest.class, "setClientServerObserverForBeforeInterestRecovery");
       vm2.invoke(ConflationDUnitTest.class, "setAllCountersZero");
       vm2.invoke(ConflationDUnitTest.class, "assertAllCountersZero");
       vm2.invoke(ConflationDUnitTest.class, "registerInterest");
@@ -484,10 +484,10 @@ public class ConflationDUnitTest extends DistributedTestCase
    * reset all counters to zero before interest recovery
    *
    */
-  public static void setBridgeObserverForBeforeInterestRecovery()
+  public static void setClientServerObserverForBeforeInterestRecovery()
   {
     PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = true;
-    BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       public void beforeInterestRecovery()
       {
         setAllCountersZero();
@@ -648,7 +648,7 @@ public class ConflationDUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME1, attrs);
     cache.createRegion(REGION_NAME2, attrs);
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
     server.setPort(port);
     server.setNotifyBySubscription(true);
@@ -793,8 +793,8 @@ public class ConflationDUnitTest extends DistributedTestCase
   public static void getStatsOnServer()
   {
     Cache cache = GemFireCacheImpl.getInstance();
-    Iterator itr = cache.getBridgeServers().iterator();
-    BridgeServerImpl server = (BridgeServerImpl)itr.next();
+    Iterator itr = cache.getCacheServers().iterator();
+    CacheServerImpl server = (CacheServerImpl)itr.next();
     Iterator iter_prox = server.getAcceptor().getCacheClientNotifier()
         .getClientProxies().iterator();
     int ccpCount=0;


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

Posted by bs...@apache.org.
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){          



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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerAdvisor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerAdvisor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerAdvisor.java
new file mode 100644
index 0000000..646f336
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerAdvisor.java
@@ -0,0 +1,164 @@
+/*=========================================================================
+ * Copyright (c) 2002-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
+ * more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+
+package com.gemstone.gemfire.internal.cache;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.server.ServerLoad;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisee;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
+
+
+/**
+ * Used to give advice to a cache server.
+ * Cache server currently need to know about controller's
+ * @author darrel
+ *
+ */
+public class CacheServerAdvisor extends GridAdvisor {
+  
+  private CacheServerAdvisor(DistributionAdvisee server) {
+    super(server);
+  }
+
+  public static CacheServerAdvisor createCacheServerAdvisor(DistributionAdvisee server) {
+    CacheServerAdvisor advisor = new CacheServerAdvisor(server);
+    advisor.initialize();
+    return advisor;
+  }
+
+  @Override
+  public String toString() {
+    return "CacheServerAdvisor for " + getAdvisee().getFullPath();
+  }
+
+  /** Instantiate new distribution profile for this member */
+  @Override
+  protected Profile instantiateProfile(
+      InternalDistributedMember memberId, int version) {
+    return new CacheServerProfile(memberId, version);
+  }
+  
+  /**
+   * Describes a cache server for distribution purposes.
+   */
+  public static class CacheServerProfile extends GridAdvisor.GridProfile {
+    private String[] groups;
+    private int maxConnections;
+    private ServerLoad initialLoad;
+    private long loadPollInterval;
+
+    /** for internal use, required for DataSerializer.readObject */
+    public CacheServerProfile() {
+    }
+
+    public CacheServerProfile(InternalDistributedMember memberId, int version) {
+      super(memberId, version);
+    }
+
+    public CacheServerProfile(CacheServerProfile toCopy) {
+      super(toCopy);
+      this.groups = toCopy.groups;
+    }
+
+    /** don't modify the returned array! */
+    public String[] getGroups() {
+      return this.groups;
+    }
+    public void setGroups(String[] groups) {
+      this.groups = groups;
+    }
+    
+    public ServerLoad getInitialLoad() {
+      return initialLoad;
+    }
+    
+    public int getMaxConnections() {
+      return maxConnections;
+    }
+    
+    public void setMaxConnections(int maxConnections) {
+      this.maxConnections = maxConnections;
+    }
+
+    public void setInitialLoad(ServerLoad initialLoad) {
+      this.initialLoad = initialLoad;
+    }
+    public long getLoadPollInterval() {
+      return this.loadPollInterval;
+    }
+    public void setLoadPollInterval(long v) {
+      this.loadPollInterval = v;
+    }
+
+    /**
+     * Used to process an incoming cache server profile. Any controller in this
+     * vm needs to be told about this incoming new cache server. The reply
+     * needs to contain any controller(s) that exist in this vm.
+     * 
+     * @since 5.7
+     */
+    @Override
+    public void processIncoming(DistributionManager dm, String adviseePath,
+        boolean removeProfile, boolean exchangeProfiles,
+        final List<Profile> replyProfiles) {
+      // tell local controllers about this cache server
+      tellLocalControllers(removeProfile, exchangeProfiles, replyProfiles);
+      // for QRM messaging we need cache servers to know about each other
+      tellLocalBridgeServers(removeProfile, exchangeProfiles, replyProfiles);
+    }
+
+    @Override
+    public int getDSFID() {
+      return CACHE_SERVER_PROFILE;
+    }
+
+    @Override
+    public void toData(DataOutput out) throws IOException {
+      super.toData(out);
+      DataSerializer.writeStringArray(this.groups, out);
+      out.writeInt(maxConnections);
+      InternalDataSerializer.invokeToData(initialLoad, out);
+      out.writeLong(getLoadPollInterval());
+    }
+
+    @Override
+    public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+      super.fromData(in);
+      this.groups = DataSerializer.readStringArray(in);
+      this.maxConnections = in.readInt();
+      this.initialLoad = new ServerLoad();
+      InternalDataSerializer.invokeFromData(initialLoad, in);
+      setLoadPollInterval(in.readLong());
+    }
+
+    @Override
+    public StringBuilder getToStringHeader() {
+      return new StringBuilder("BridgeServerProfile");
+    }
+
+    @Override
+    public void fillInToString(StringBuilder sb) {
+      super.fillInToString(sb);
+      if (this.groups != null) {
+        sb.append("; groups=" + Arrays.asList(this.groups));
+        sb.append("; maxConnections=" + maxConnections);
+        sb.append("; initialLoad=" + initialLoad);
+        sb.append("; loadPollInterval=" + getLoadPollInterval());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
new file mode 100644
index 0000000..422711e
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
@@ -0,0 +1,812 @@
+/*=========================================================================
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.GemFireIOException;
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.InvalidValueException;
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.ClientSession;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.DiskStore;
+import com.gemstone.gemfire.cache.DiskStoreFactory;
+import com.gemstone.gemfire.cache.DynamicRegionFactory;
+import com.gemstone.gemfire.cache.EvictionAction;
+import com.gemstone.gemfire.cache.ExpirationAction;
+import com.gemstone.gemfire.cache.ExpirationAttributes;
+import com.gemstone.gemfire.cache.InterestRegistrationListener;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionExistsException;
+import com.gemstone.gemfire.cache.Scope;
+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.server.internal.LoadMonitor;
+import com.gemstone.gemfire.cache.wan.GatewayTransportFilter;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisee;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisor;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisor.Profile;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.ResourceEvent;
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.OSProcess;
+import com.gemstone.gemfire.internal.admin.ClientHealthMonitoringRegion;
+import com.gemstone.gemfire.internal.cache.CacheServerAdvisor.CacheServerProfile;
+import com.gemstone.gemfire.internal.cache.ha.HARegionQueue;
+import com.gemstone.gemfire.internal.cache.tier.Acceptor;
+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.ClientProxyMembershipID;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.management.membership.ClientMembership;
+import com.gemstone.gemfire.management.membership.ClientMembershipListener;
+
+/**
+ * An implementation of the <code>CacheServer</code> interface that delegates
+ * most of the heavy lifting to an {@link Acceptor}.
+ * 
+ * @author David Whitlock
+ * @since 4.0
+ */
+@SuppressWarnings("deprecation")
+public class CacheServerImpl
+  extends AbstractCacheServer
+  implements DistributionAdvisee {
+
+  private static final Logger logger = LogService.getLogger();
+  
+  private static final int FORCE_LOAD_UPDATE_FREQUENCY= Integer.getInteger("gemfire.BridgeServer.FORCE_LOAD_UPDATE_FREQUENCY", 10).intValue();
+  
+  /** The acceptor that does the actual serving */
+  private volatile AcceptorImpl acceptor;
+
+  /**
+   * The advisor used by this cache server.
+   * @since 5.7
+   */
+  private volatile CacheServerAdvisor advisor;
+
+  /**
+   * The monitor used to monitor load on this
+   * bridge server and distribute load to the locators
+   * @since 5.7
+   */
+  private volatile LoadMonitor loadMonitor;
+
+  /**
+   * boolean that represents whether this server is a GatewayReceiver or a simple BridgeServer
+   */
+  private boolean isGatewayReceiver;
+  
+  private List<GatewayTransportFilter> gatewayTransportFilters = Collections.EMPTY_LIST;
+  
+  /**
+   * Needed because this guy is an advisee
+   * @since 5.7
+   */
+  private int serialNumber; // changed on each start
+
+  public static final boolean ENABLE_NOTIFY_BY_SUBSCRIPTION_FALSE = 
+  Boolean.getBoolean("gemfire.cache-server.enable-notify-by-subscription-false");
+  
+ 
+  // ////////////////////// Constructors //////////////////////
+
+  /**
+   * Creates a new <code>BridgeServerImpl</code> that serves the contents of
+   * the give <code>Cache</code>. It has the default configuration.
+   */
+  public CacheServerImpl(GemFireCacheImpl cache, boolean isGatewayReceiver) {
+    super(cache);
+    this.isGatewayReceiver = isGatewayReceiver;
+  }
+
+  // //////////////////// Instance Methods ///////////////////
+  
+  public CancelCriterion getCancelCriterion() {
+    return cache.getCancelCriterion();    
+  }
+
+  /**
+   * Checks to see whether or not this bridge server is running. If so, an
+   * {@link IllegalStateException} is thrown.
+   */
+  private void checkRunning() {
+    if (this.isRunning()) {
+      throw new IllegalStateException(LocalizedStrings.CacheServerImpl_A_CACHE_SERVERS_CONFIGURATION_CANNOT_BE_CHANGED_ONCE_IT_IS_RUNNING.toLocalizedString());
+    }
+  }
+
+  public boolean isGatewayReceiver() {
+    return this.isGatewayReceiver;
+  }
+  
+  @Override
+  public int getPort() {
+    if (this.acceptor != null) {
+      return this.acceptor.getPort();
+    }
+    else {
+      return super.getPort();
+    }
+  }
+
+  @Override
+  public void setPort(int port) {
+    checkRunning();
+    super.setPort(port);
+  }
+
+  @Override
+  public void setBindAddress(String address) {
+    checkRunning();
+    super.setBindAddress(address);
+  }
+  @Override
+  public void setHostnameForClients(String name) {
+    checkRunning();
+    super.setHostnameForClients(name);
+  }
+
+  @Override
+  public void setMaxConnections(int maxCon) {
+    checkRunning();
+    super.setMaxConnections(maxCon);
+  }
+
+  @Override
+  public void setMaxThreads(int maxThreads) {
+    checkRunning();
+    super.setMaxThreads(maxThreads);
+  }
+
+  @Override
+  public void setNotifyBySubscription(boolean b) {
+    checkRunning();
+    if (CacheServerImpl.ENABLE_NOTIFY_BY_SUBSCRIPTION_FALSE) {
+      this.notifyBySubscription = b;
+    }
+  }
+
+  @Override
+  public void setMaximumMessageCount(int maximumMessageCount) {
+    checkRunning();
+    super.setMaximumMessageCount(maximumMessageCount);
+  }
+
+  @Override
+  public void setSocketBufferSize(int socketBufferSize) {
+    this.socketBufferSize = socketBufferSize;
+  }
+
+  @Override
+  public int getSocketBufferSize() {
+    return this.socketBufferSize;
+  }
+  
+  @Override
+  public void setMaximumTimeBetweenPings(int maximumTimeBetweenPings) {
+    this.maximumTimeBetweenPings = maximumTimeBetweenPings;
+  }
+
+  @Override
+  public int getMaximumTimeBetweenPings() {
+    return this.maximumTimeBetweenPings;
+  }
+
+
+  @Override
+  public void setLoadPollInterval(long loadPollInterval) {
+    checkRunning();
+    super.setLoadPollInterval(loadPollInterval);
+  }
+
+  @Override
+  public int getMaximumMessageCount() {
+    return this.maximumMessageCount;
+  }
+
+  @Override
+  public void setLoadProbe(ServerLoadProbe loadProbe) {
+    checkRunning();
+    super.setLoadProbe(loadProbe);
+  }
+
+  public void setGatewayTransportFilter(
+      List<GatewayTransportFilter> transportFilters) {
+    this.gatewayTransportFilters = transportFilters;
+  }
+  
+  @Override
+  public int getMessageTimeToLive() {
+    return this.messageTimeToLive;
+  }
+  
+
+  public ClientSubscriptionConfig getClientSubscriptionConfig(){
+    return this.clientSubscriptionConfig;
+  }
+
+  /**
+   * Sets the configuration of <b>this</b> <code>CacheServer</code> based on
+   * the configuration of <b>another</b> <code>CacheServer</code>.
+   */
+  public void configureFrom(CacheServer other) {
+    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());
+    }
+  }
+
+  @Override
+  public synchronized void start() throws IOException {
+    Assert.assertTrue(this.cache != null);
+    boolean isSqlFabricSystem = ((GemFireCacheImpl)this.cache).isSqlfSystem();
+    
+    this.serialNumber = createSerialNumber();
+    if (DynamicRegionFactory.get().isOpen()) {
+      // force notifyBySubscription to be true so that meta info is pushed
+      // from servers to clients instead of invalidates.
+      if (!this.notifyBySubscription) {
+        logger.info(LocalizedMessage.create(LocalizedStrings.CacheServerImpl_FORCING_NOTIFYBYSUBSCRIPTION_TO_SUPPORT_DYNAMIC_REGIONS));
+        this.notifyBySubscription = true;
+      }
+    }
+    this.advisor = CacheServerAdvisor.createCacheServerAdvisor(this);
+    this.loadMonitor = new LoadMonitor(loadProbe, maxConnections,
+        loadPollInterval, FORCE_LOAD_UPDATE_FREQUENCY, 
+        advisor);
+    List overflowAttributesList = new LinkedList();
+    ClientSubscriptionConfig csc = this.getClientSubscriptionConfig();
+    overflowAttributesList.add(0, csc.getEvictionPolicy());
+    overflowAttributesList.add(1, Integer.valueOf(csc.getCapacity()));
+    overflowAttributesList.add(2, Integer.valueOf(this.port));
+    String diskStoreName = csc.getDiskStoreName();
+    if (diskStoreName != null) {
+      overflowAttributesList.add(3, diskStoreName);
+      overflowAttributesList.add(4, true); // indicator to use diskstore
+    } else {
+      overflowAttributesList.add(3, csc.getOverflowDirectory());
+      overflowAttributesList.add(4, false);
+    }
+
+    this.acceptor = new AcceptorImpl(getPort(), 
+                                     getBindAddress(),
+                                     getNotifyBySubscription(),
+                                     getSocketBufferSize(), 
+                                     getMaximumTimeBetweenPings(), 
+                                     this.cache,
+                                     getMaxConnections(), 
+                                     getMaxThreads(), 
+                                     getMaximumMessageCount(),
+                                     getMessageTimeToLive(),
+                                     getTransactionTimeToLive(),
+                                     this.loadMonitor,
+                                     overflowAttributesList, 
+                                     isSqlFabricSystem,
+                                     this.isGatewayReceiver,
+                                     this.gatewayTransportFilters, this.tcpNoDelay);
+
+    this.acceptor.start();
+    this.advisor.handshake();
+    this.loadMonitor.start(new ServerLocation(getExternalAddress(),
+        getPort()), acceptor.getStats());
+    
+    // TODO : Need to provide facility to enable/disable client health monitoring.
+    //Creating ClientHealthMonitoring region.
+    // Force initialization on current cache
+    if(cache instanceof GemFireCacheImpl) {
+      ClientHealthMonitoringRegion.getInstance((GemFireCacheImpl)cache);
+    }
+    this.cache.getLoggerI18n().config(LocalizedStrings.CacheServerImpl_CACHESERVER_CONFIGURATION___0, getConfig());
+    
+    /* 
+     * If the stopped bridge server is restarted, we'll need to re-register the 
+     * client membership listener. If the listener is already registered it 
+     * won't be registered as would the case when start() is invoked for the 
+     * first time.  
+     */
+    ClientMembershipListener[] membershipListeners = 
+                                ClientMembership.getClientMembershipListeners();
+    
+    boolean membershipListenerRegistered = false;
+    for (ClientMembershipListener membershipListener : membershipListeners) {
+      //just checking by reference as the listener instance is final
+      if (listener == membershipListener) {
+        membershipListenerRegistered = true;
+        break;
+      }
+    }
+    
+    if (!membershipListenerRegistered) {
+      ClientMembership.registerClientMembershipListener(listener);
+    }
+    
+    if (!isGatewayReceiver) {
+      InternalDistributedSystem system = ((GemFireCacheImpl) this.cache)
+          .getDistributedSystem();
+      system.handleResourceEvent(ResourceEvent.CACHE_SERVER_START, this);
+    }
+    
+  }
+
+  
+  /**
+   * Gets the address that this bridge server can be contacted on from external
+   * processes.
+   * @since 5.7
+   */
+  public String getExternalAddress() {
+    return getExternalAddress(true);
+  }
+  
+  public String getExternalAddress(boolean checkServerRunning) {
+    if (checkServerRunning) {
+      if (!this.isRunning()) {
+        String s = "A bridge server's bind address is only available if it has been started";
+        this.cache.getCancelCriterion().checkCancelInProgress(null);
+        throw new IllegalStateException(s);
+      }
+    }
+    if (this.hostnameForClients == null || this.hostnameForClients.equals("")) {
+      if (this.acceptor != null) {
+        return this.acceptor.getExternalAddress();
+      }
+      else {
+        return null;
+      }
+    }
+    else {
+      return this.hostnameForClients;
+    }
+  }
+
+  public boolean isRunning() {
+    return this.acceptor != null && this.acceptor.isRunning();
+  }
+
+  public synchronized void stop() {
+    if (!isRunning()) {
+      return;
+    }
+    
+    RuntimeException firstException = null;
+    
+    try {
+      if(this.loadMonitor != null) {
+        this.loadMonitor.stop();
+      }
+    } catch(RuntimeException e) {
+      cache.getLoggerI18n().warning(LocalizedStrings.CacheServerImpl_CACHESERVER_ERROR_CLOSING_LOAD_MONITOR, e);
+      firstException = e;
+    }
+    
+    try {
+      if (this.advisor != null) {
+        this.advisor.close();
+      }
+    } catch(RuntimeException e) {
+      cache.getLoggerI18n().warning(LocalizedStrings.CacheServerImpl_CACHESERVER_ERROR_CLOSING_ADVISOR, e);
+      firstException = e;
+    } 
+    
+    try {
+      if (this.acceptor != null) {
+        this.acceptor.close();
+      }
+    } catch(RuntimeException e) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.CacheServerImpl_CACHESERVER_ERROR_CLOSING_ACCEPTOR_MONITOR), e);
+      if (firstException != null) {
+        firstException = e;
+      }
+    }
+    
+    if(firstException != null) {
+      throw firstException;
+    }
+    
+    //TODO : We need to clean up the admin region created for client
+    //monitoring.
+    
+    // BridgeServer is still available, just not running, so we don't take
+    // it out of the cache's list...
+    // cache.removeBridgeServer(this);
+
+    /* Assuming start won't be called after stop */
+    ClientMembership.unregisterClientMembershipListener(listener);
+    
+    TXManagerImpl txMgr = (TXManagerImpl) cache.getCacheTransactionManager();
+    txMgr.removeHostedTXStatesForClients();
+    
+    if (!isGatewayReceiver) {
+      InternalDistributedSystem system = ((GemFireCacheImpl) this.cache)
+          .getDistributedSystem();
+      system.handleResourceEvent(ResourceEvent.CACHE_SERVER_STOP, this);
+    }
+
+  }
+
+  private String getConfig() {
+    ClientSubscriptionConfig csc = this.getClientSubscriptionConfig();
+    String str =
+    "port=" + getPort() + " max-connections=" + getMaxConnections()
+        + " max-threads=" + getMaxThreads() + " notify-by-subscription="
+        + getNotifyBySubscription() + " socket-buffer-size="
+        + getSocketBufferSize() + " maximum-time-between-pings="
+        + getMaximumTimeBetweenPings() + " maximum-message-count="
+        + getMaximumMessageCount() + " message-time-to-live="
+        + getMessageTimeToLive() + " eviction-policy=" + csc.getEvictionPolicy()
+        + " capacity=" + csc.getCapacity() + " overflow directory=";
+    if (csc.getDiskStoreName() != null) {
+      str += csc.getDiskStoreName();
+    } else {
+      str += csc.getOverflowDirectory(); 
+    }
+    str += 
+        " groups=" + Arrays.asList(getGroups())
+        + " loadProbe=" + loadProbe
+        + " loadPollInterval=" + loadPollInterval
+        + " tcpNoDelay=" + tcpNoDelay;
+    return str;
+  }
+
+  @Override
+  public String toString() {
+    ClientSubscriptionConfig csc = this.getClientSubscriptionConfig();
+    String str = 
+    "CacheServer on port=" + getPort() + " client subscription config policy="
+        + csc.getEvictionPolicy() + " client subscription config capacity="
+        + csc.getCapacity();
+    if (csc.getDiskStoreName() != null) {
+      str += " client subscription config overflow disk store="
+        + csc.getDiskStoreName();
+    } else {
+      str += " client subscription config overflow directory="
+        + csc.getOverflowDirectory();
+    }
+    return str;
+  }
+
+  /**
+   * Test method used to access the internal acceptor
+   * 
+   * @return the internal acceptor
+   */
+  public AcceptorImpl getAcceptor() {
+    return this.acceptor;
+  }
+
+  // DistributionAdvisee methods
+
+  public DM getDistributionManager() {
+    return getSystem().getDistributionManager();
+  }
+  
+  public ClientSession getClientSession(String durableClientId) {
+    return getCacheClientNotifier().getClientProxy(durableClientId);
+  }
+
+  public ClientSession getClientSession(DistributedMember member) {
+    return getCacheClientNotifier().getClientProxy(
+        ClientProxyMembershipID.getClientId(member));
+  }
+  
+  public Set getAllClientSessions() {
+    return new HashSet(getCacheClientNotifier().getClientProxies());
+  }
+
+  /**
+   * create client subscription
+   * 
+   * @param cache
+   * @param ePolicy
+   * @param capacity
+   * @param port
+   * @param overFlowDir
+   * @param isDiskStore
+   * @return client subscription name
+   * @since 5.7
+   */
+  public static String clientMessagesRegion(GemFireCacheImpl cache, String ePolicy,
+      int capacity, int port, String overFlowDir, boolean isDiskStore) {
+    AttributesFactory factory = getAttribFactoryForClientMessagesRegion(cache, 
+        ePolicy, capacity, overFlowDir, isDiskStore);
+    RegionAttributes attr = factory.create();
+
+    return createClientMessagesRegion(attr, cache, capacity, port);
+  }
+
+  public static AttributesFactory getAttribFactoryForClientMessagesRegion(
+      GemFireCacheImpl cache,
+      String ePolicy, int capacity, String overflowDir, boolean isDiskStore)
+      throws InvalidValueException, GemFireIOException {
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+
+    if (isDiskStore) {
+      // overflowDir parameter is actually diskstore name
+      factory.setDiskStoreName(overflowDir);
+      // client subscription queue is always overflow to disk, so do async
+      // see feature request #41479
+      factory.setDiskSynchronous(true);
+    } else if  (overflowDir == null || overflowDir.equals(ClientSubscriptionConfig.DEFAULT_OVERFLOW_DIRECTORY)) {
+      factory.setDiskStoreName(null);
+      // client subscription queue is always overflow to disk, so do async
+      // see feature request #41479
+      factory.setDiskSynchronous(true);
+    } else {
+      File dir = new File(overflowDir + File.separatorChar
+          + generateNameForClientMsgsRegion(OSProcess.getId()));
+      // This will delete the overflow directory when virtual machine terminates.
+      dir.deleteOnExit();
+      if (!dir.mkdirs() && !dir.isDirectory()) {
+        throw new GemFireIOException("Could not create client subscription overflow directory: "
+            + dir.getAbsolutePath());
+      }
+      File[] dirs = { dir };
+      DiskStoreFactory dsf = cache.createDiskStoreFactory();
+      DiskStore bsi = dsf.setAutoCompact(true)
+      .setDiskDirsAndSizes(dirs, new int[] { Integer.MAX_VALUE })
+      .create("bsi");
+      factory.setDiskStoreName("bsi");
+      // backward compatibility, it was sync
+      factory.setDiskSynchronous(true);
+    }
+    factory.setDataPolicy(DataPolicy.NORMAL);
+    // enable statistics
+    factory.setStatisticsEnabled(true);
+    /* setting LIFO related eviction attributes */
+    if (HARegionQueue.HA_EVICTION_POLICY_ENTRY.equals(ePolicy)) {
+      factory
+          .setEvictionAttributes(EvictionAttributesImpl.createLIFOEntryAttributes(
+              capacity, EvictionAction.OVERFLOW_TO_DISK));
+    }
+    else if (HARegionQueue.HA_EVICTION_POLICY_MEMORY.equals(ePolicy)) { // condition refinement
+      factory
+          .setEvictionAttributes(EvictionAttributesImpl.createLIFOMemoryAttributes(
+              capacity, EvictionAction.OVERFLOW_TO_DISK));
+    }
+    else {
+      // throw invalid eviction policy exception
+      throw new InvalidValueException(
+        LocalizedStrings.CacheServerImpl__0_INVALID_EVICTION_POLICY.toLocalizedString(ePolicy));
+    }
+    return factory;
+  }
+
+  public static String createClientMessagesRegion(RegionAttributes attr,
+      GemFireCacheImpl cache, int capacity, int port) {
+    // generating unique name in VM for ClientMessagesRegion
+    String regionName = generateNameForClientMsgsRegion(port);
+    try {
+      cache.createVMRegion(regionName, attr,
+          new InternalRegionArguments().setDestroyLockFlag(true)
+              .setRecreateFlag(false).setSnapshotInputStream(null)
+              .setImageTarget(null).setIsUsedForMetaRegion(true));
+    }
+    catch (RegionExistsException ree) {
+      InternalGemFireError assErr = new InternalGemFireError(
+          "unexpected exception");
+      assErr.initCause(ree);
+      throw assErr;
+    }
+    catch (IOException e) {
+      // only if loading snapshot, not here
+      InternalGemFireError assErr = new InternalGemFireError(
+          "unexpected exception");
+      assErr.initCause(e);
+      throw assErr;
+    }
+    catch (ClassNotFoundException e) {
+      // only if loading snapshot, not here
+      InternalGemFireError assErr = new InternalGemFireError(
+          "unexpected exception");
+      assErr.initCause(e);
+      throw assErr;
+    }
+    return regionName;
+  }
+
+  public static String createClientMessagesRegionForTesting(GemFireCacheImpl cache,
+      String ePolicy, int capacity, int port, int expiryTime, String overFlowDir, boolean isDiskStore) {
+    AttributesFactory factory = getAttribFactoryForClientMessagesRegion(cache, 
+        ePolicy, capacity, overFlowDir, isDiskStore);
+    ExpirationAttributes ea = new ExpirationAttributes(expiryTime,
+        ExpirationAction.LOCAL_INVALIDATE);
+    factory.setEntryTimeToLive(ea);
+    RegionAttributes attr = factory.create();
+
+    return createClientMessagesRegion(attr, cache, capacity, port);
+  }
+
+  /**
+   * Generates the name for the client subscription using the given id.
+   * 
+   * @param id
+   * @return String
+   * @since 5.7 
+   */
+  public static String generateNameForClientMsgsRegion(int id) {
+    return ClientSubscriptionConfigImpl.CLIENT_SUBSCRIPTION + "_" + id;
+  }
+
+  /*
+   * Marker class name to identify the lock more easily in thread dumps private
+   * static class ClientMessagesRegionLock extends Object { }
+   */
+  public DistributionAdvisor getDistributionAdvisor() {
+    return this.advisor;
+  }
+  
+  /**
+   * Returns the BridgeServerAdvisor for this server
+   */
+  public CacheServerAdvisor getCacheServerAdvisor() {
+    return this.advisor;
+  }
+  
+  public Profile getProfile() {
+    return getDistributionAdvisor().createProfile();
+  }
+  
+  public DistributionAdvisee getParentAdvisee() {
+    return null;
+  }
+  
+  /**
+   * Returns the underlying <code>InternalDistributedSystem</code> connection.
+   * @return the underlying <code>InternalDistributedSystem</code>
+   */
+  public InternalDistributedSystem getSystem() {
+    return (InternalDistributedSystem)this.cache.getDistributedSystem();
+  }
+  
+  public String getName() {
+    return "CacheServer";
+  }
+  
+  public String getFullPath() {
+    return getName();
+  }
+
+  private final static AtomicInteger profileSN = new AtomicInteger();
+  
+  private static int createSerialNumber() {
+    return profileSN.incrementAndGet();
+  }
+
+  /**
+   * Returns an array of all the groups of this bridge server.
+   * This includes those from the groups gemfire property
+   * and those explicitly added to this server.
+   */
+  public String[] getCombinedGroups() {
+    ArrayList<String> groupList = new ArrayList<String>();
+    for (String g: MemberAttributes.parseGroups(null, getSystem().getConfig().getGroups())) {
+      if (!groupList.contains(g)) {
+        groupList.add(g);
+      }
+    }
+    for (String g: getGroups()) {
+      if (!groupList.contains(g)) {
+        groupList.add(g);
+      }
+    }
+    String[] groups = new String[groupList.size()];
+    return groupList.toArray(groups);
+  }
+  
+  public /*synchronized causes deadlock*/ void fillInProfile(Profile profile) {
+    assert profile instanceof CacheServerProfile;
+    CacheServerProfile bp = (CacheServerProfile)profile;
+    bp.setHost(getExternalAddress(false));
+    bp.setPort(getPort());
+    bp.setGroups(getCombinedGroups());
+    bp.setMaxConnections(maxConnections);
+    bp.setInitialLoad(loadMonitor.getLastLoad());
+    bp.setLoadPollInterval(getLoadPollInterval());
+    bp.serialNumber = getSerialNumber();
+    bp.finishInit();
+  }
+
+  public int getSerialNumber() {
+    return this.serialNumber;
+  }
+
+  
+   protected CacheClientNotifier getCacheClientNotifier() {
+    return getAcceptor().getCacheClientNotifier();
+  } 
+   
+  /**
+   * Registers a new <code>InterestRegistrationListener</code> with the set of
+   * <code>InterestRegistrationListener</code>s.
+   * 
+   * @param listener
+   *                The <code>InterestRegistrationListener</code> to register
+   * @throws IllegalStateException if the BridgeServer has not been started
+   * @since 5.8Beta
+   */
+  public void registerInterestRegistrationListener(
+      InterestRegistrationListener listener) {
+    if (!this.isRunning()) {
+      throw new IllegalStateException(LocalizedStrings.CacheServerImpl_MUST_BE_RUNNING.toLocalizedString());
+    }
+    getCacheClientNotifier().registerInterestRegistrationListener(listener); 
+  }
+
+  /**
+   * Unregisters an existing <code>InterestRegistrationListener</code> from
+   * the set of <code>InterestRegistrationListener</code>s.
+   * 
+   * @param listener
+   *                The <code>InterestRegistrationListener</code> to
+   *                unregister
+   * 
+   * @since 5.8Beta
+   */
+  public void unregisterInterestRegistrationListener(
+      InterestRegistrationListener listener) {
+    getCacheClientNotifier().unregisterInterestRegistrationListener(listener);     
+  }
+
+  /**
+   * Returns a read-only set of <code>InterestRegistrationListener</code>s
+   * registered with this notifier.
+   * 
+   * @return a read-only set of <code>InterestRegistrationListener</code>s
+   *         registered with this notifier
+   * 
+   * @since 5.8Beta
+   */
+  public Set getInterestRegistrationListeners() {
+    return getCacheClientNotifier().getInterestRegistrationListeners(); 
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientRegionEventImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientRegionEventImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientRegionEventImpl.java
new file mode 100755
index 0000000..176ddcb
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientRegionEventImpl.java
@@ -0,0 +1,108 @@
+/*=========================================================================
+ * Copyright (c) 2002-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
+ * more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+
+package com.gemstone.gemfire.internal.cache;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
+
+//import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
+
+/**
+ * Class <code>ClientRegionEventImpl</code> is a
+ * region event with the client's
+ * host and port for notification purposes.
+ * 
+ * @author Girish Thombare
+ * 
+ * @since 5.1
+ */
+public final class ClientRegionEventImpl extends RegionEventImpl
+  {
+
+  /**
+   * The originating membershipId of this event.
+   */
+  private  ClientProxyMembershipID context;
+
+  public ClientRegionEventImpl() {
+  }
+  
+  /**
+   * To be called from the Distributed Message without setting EventID
+   * @param region
+   * @param op
+   * @param callbackArgument
+   * @param originRemote
+   * @param distributedMember
+   */
+  public ClientRegionEventImpl(LocalRegion region, Operation op, Object callbackArgument,boolean originRemote, DistributedMember distributedMember,ClientProxyMembershipID contx) {
+    super(region, op,callbackArgument, originRemote,distributedMember);
+    setContext(contx);
+  }
+
+  public ClientRegionEventImpl(LocalRegion region, Operation op, Object callbackArgument,boolean originRemote, DistributedMember distributedMember,ClientProxyMembershipID contx,EventID eventId) {
+      super(region, op,callbackArgument, originRemote,distributedMember, eventId);
+      setContext(contx);
+  }
+
+
+  /**
+   * sets The membershipId originating this event
+   *  
+   */
+  protected void setContext(ClientProxyMembershipID contx)
+  {
+    this.context = contx;
+  }
+
+  /**
+   * Returns The context originating this event
+   * 
+   * @return The context originating this event
+   */
+  @Override
+  public ClientProxyMembershipID getContext()
+  {
+    return this.context;
+  }
+
+  @Override
+  public String toString()
+  {
+    String superStr = super.toString();
+    StringBuffer buffer = new StringBuffer();
+    String str = superStr.substring(0, superStr.length() - 1);
+    buffer.append(str).append(";context=").append(getContext()).append(']');
+    return buffer.toString();
+  }
+
+  @Override
+  public int getDSFID() {
+    return CLIENT_REGION_EVENT;
+  }
+
+  @Override
+  public void toData(DataOutput out) throws IOException
+  {
+    super.toData(out);
+    DataSerializer.writeObject(getContext(), out);
+  }
+
+  @Override
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException
+  {
+    super.fromData(in);
+    setContext(ClientProxyMembershipID.readCanonicalized(in));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserver.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserver.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserver.java
new file mode 100755
index 0000000..0646f04
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserver.java
@@ -0,0 +1,90 @@
+/*=========================================================================
+ * 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;
+
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+
+/**
+ * This interface is used by testing/debugging code to be notified of different
+ * client/server events.
+ * See the documentation for class ClientServerObserverHolder for details.
+ * 
+ * @author Yogesh Mahajan
+ * @since 5.1
+ *  
+ */
+public interface ClientServerObserver
+{
+  /**
+   * This callback is called when now primary Ep is identified.
+   */
+  public void afterPrimaryIdentificationFromBackup(ServerLocation location);
+
+  /**
+   * This callback is called just before interest registartion
+   */
+  public void beforeInterestRegistration();
+
+  /**
+   * This callback is called just after interest registartion
+   */
+  public void afterInterestRegistration();
+
+  /**
+   * This callback is called just before primary identification
+   */
+  public void beforePrimaryIdentificationFromBackup();
+
+  /**
+   * This callback is called just before Interest Recovery by DSM thread happens
+   */
+  public void beforeInterestRecovery();
+  
+  /**
+   * Invoked by CacheClientUpdater just before invoking endpointDied for
+   * fail over
+   * @param location ServerLocation which has failed
+   */
+  public void beforeFailoverByCacheClientUpdater(ServerLocation location);
+  /**
+   * Invoked before sending an instantiator message to server
+   * 
+   * @param eventId
+   */
+  public void beforeSendingToServer(EventID eventId);
+  /**
+   * Invoked after sending an instantiator message to server 
+   * 
+   * @param eventId
+   */
+  public void afterReceivingFromServer(EventID eventId);
+
+  /**
+   * This callback is called just before sending client ack to the primary servrer.
+   */
+   public void beforeSendingClientAck();  
+
+   /**
+    * Invoked after Message is created
+    *
+    * @param msg
+    */
+   public void afterMessageCreation(Message msg);
+   
+   /**
+    * Invoked after Queue Destroy Message has been sent
+    */
+   public void afterQueueDestroyMessage();
+   
+   /**
+    * Invoked after a primary is recovered from a backup or new connection. 
+    */
+   public void afterPrimaryRecovered(ServerLocation location);
+   
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserverAdapter.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserverAdapter.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserverAdapter.java
new file mode 100755
index 0000000..094bb58
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserverAdapter.java
@@ -0,0 +1,107 @@
+/*=========================================================================
+ * 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;
+
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+
+/**
+ * This class provides 'do-nothing' implementations of all of the methods of
+ * interface ClientServerObserver. See the documentation for class
+ * ClientServerObserverHolder for details.
+ * 
+ * @author Yogesh Mahajan
+ * @since 5.1
+ */
+public class ClientServerObserverAdapter implements ClientServerObserver
+{
+  /**
+   * This callback is called when now primary Ep is identified.
+   */
+  public void afterPrimaryIdentificationFromBackup(ServerLocation primaryEndpoint)
+  {
+  }
+
+  /**
+   * This callback is called just before interest registartion
+   */
+  public void beforeInterestRegistration()
+  {
+  }
+
+  /**
+   * This callback is called just after interest registartion
+   */
+  public void afterInterestRegistration()
+  {
+  }
+
+  /**
+   * This callback is called just before primary identification
+   */
+  public void beforePrimaryIdentificationFromBackup()
+  {
+  }
+
+  /**
+   * This callback is called just before Interest Recovery by DSM thread happens
+   */
+  public void beforeInterestRecovery()
+  {
+
+  }
+
+  public void beforeFailoverByCacheClientUpdater(ServerLocation epFailed)
+  {
+  }
+  /**
+   * Invoked before sending an instantiator message to server
+   * 
+   * @param eventId
+   */
+  public void beforeSendingToServer(EventID eventId){
+    
+  }
+  /**
+   * Invoked after sending an instantiator message to server 
+   * 
+   * @param eventId
+   */
+  public void afterReceivingFromServer(EventID eventId){
+    
+  }
+  
+  /**
+   * This callback is called just before sending client ack to the primary servrer.
+   */
+  public void beforeSendingClientAck(){
+    
+  }  
+
+  /**
+   * Invoked after Message is created
+   *
+   * @param msg
+   */
+  public void afterMessageCreation(Message msg){
+  
+  }
+  
+  /**
+   * Invoked after Queue Destroy Message has been sent
+   */
+  public void afterQueueDestroyMessage(){
+    
+  }
+  
+  /**
+   * Invoked after a primary is recovered from a backup or new connection. 
+   */
+  public void afterPrimaryRecovered(ServerLocation location) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserverHolder.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserverHolder.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserverHolder.java
new file mode 100755
index 0000000..003852b
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ClientServerObserverHolder.java
@@ -0,0 +1,53 @@
+/*=========================================================================
+ * 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;
+
+import com.gemstone.gemfire.cache.query.internal.Support;
+
+/**
+ * This class is intended to hold a single 'observer' which will receive
+ * callbacks. There can be only one such observer at a time. If no observer is
+ * needed, this member variable should point to an object with 'do-nothing'
+ * methods, such as ClientServerObserverAdapter.
+ * 
+ * @author Yogesh Mahajan
+ * @since 5.1
+ */
+public class ClientServerObserverHolder
+  {
+
+  /**
+   * The default 'do-nothing' bridge observer *
+   */
+  private static final ClientServerObserver NO_OBSERVER = new ClientServerObserverAdapter();
+
+  /**
+   * The current observer which will be notified of all query events.
+   */
+  private static ClientServerObserver _instance = NO_OBSERVER;
+
+  /**
+   * Set the given observer to be notified of events. Returns the current
+   * observer.
+   */
+  public static final ClientServerObserver setInstance(ClientServerObserver observer)
+  {
+    Support.assertArg(observer != null,
+        "setInstance expects a non-null argument!");
+    ClientServerObserver oldObserver = _instance;
+    _instance = observer;
+    return oldObserver;
+  }
+
+  /** Return the current BridgeObserver instance */
+  public static final ClientServerObserver getInstance()
+  {
+    return _instance;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyRegionOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyRegionOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyRegionOperation.java
index 921cbf9..198803d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyRegionOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyRegionOperation.java
@@ -105,9 +105,9 @@ public class DestroyRegionOperation extends DistributedCacheOperation {
   protected CacheOperationMessage createMessage()
   {
     DestroyRegionMessage mssg;
-    if (this.event instanceof BridgeRegionEventImpl) {
+    if (this.event instanceof ClientRegionEventImpl) {
       mssg = new DestroyRegionWithContextMessage();
-      ((DestroyRegionWithContextMessage)mssg).context = ((BridgeRegionEventImpl)this.event)
+      ((DestroyRegionWithContextMessage)mssg).context = ((ClientRegionEventImpl)this.event)
           .getContext();
     }
     else {
@@ -502,7 +502,7 @@ public class DestroyRegionOperation extends DistributedCacheOperation {
     @Override
     final public RegionEventImpl createRegionEvent(DistributedRegion rgn)
     {
-      BridgeRegionEventImpl event = new BridgeRegionEventImpl(rgn,
+      ClientRegionEventImpl event = new ClientRegionEventImpl(rgn,
           getOperation(), this.callbackArg, true /* originRemote */,
           getSender(), (ClientProxyMembershipID)this.context);
       return event;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java
index 4dbc9c4..8056120 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedClearOperation.java
@@ -128,9 +128,9 @@ public class DistributedClearOperation extends DistributedCacheOperation
   protected CacheOperationMessage createMessage()
   {
     ClearRegionMessage mssg;
-    if (this.event instanceof BridgeRegionEventImpl) {
+    if (this.event instanceof ClientRegionEventImpl) {
       mssg = new ClearRegionWithContextMessage();
-      ((ClearRegionWithContextMessage)mssg).context = ((BridgeRegionEventImpl)this.event)
+      ((ClearRegionWithContextMessage)mssg).context = ((ClientRegionEventImpl)this.event)
           .getContext();
 
     }
@@ -271,7 +271,7 @@ public class DistributedClearOperation extends DistributedCacheOperation
     final public RegionEventImpl createRegionEvent(DistributedRegion rgn)
     {
       
-      BridgeRegionEventImpl event = new BridgeRegionEventImpl(rgn,
+      ClientRegionEventImpl event = new ClientRegionEventImpl(rgn,
           getOperation(), this.callbackArg, true /* originRemote */,
           getSender(), (ClientProxyMembershipID)this.context);
       return event;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
index 790dc4d..77fbc88 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
@@ -2893,9 +2893,6 @@ public class DistributedRegion extends LocalRegion implements
   protected void cacheWriterChanged(CacheWriter oldWriter)
   {
     super.cacheWriterChanged(oldWriter);
-    if (isBridgeWriter(oldWriter)) {
-      oldWriter = null;
-    }
     if (oldWriter == null ^ basicGetWriter() == null) {
       new UpdateAttributesProcessor(this).distribute();
     }
@@ -2905,9 +2902,6 @@ public class DistributedRegion extends LocalRegion implements
   protected void cacheLoaderChanged(CacheLoader oldLoader)
   {
     super.cacheLoaderChanged(oldLoader);
-    if (isBridgeLoader(oldLoader)) {
-      oldLoader = null;
-    }
     if (oldLoader == null ^ basicGetLoader() == null) {
       new UpdateAttributesProcessor(this).distribute();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
index 5c428b2..357c0a8 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
@@ -28,7 +28,6 @@ import com.gemstone.gemfire.cache.EntryNotFoundException;
 import com.gemstone.gemfire.cache.ExpirationAction;
 import com.gemstone.gemfire.cache.ExpirationAttributes;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
-import com.gemstone.gemfire.cache.util.BridgeWriterException;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.PooledExecutorWithDMStats;
 import com.gemstone.gemfire.internal.SystemTimer;
@@ -364,23 +363,6 @@ public abstract class ExpiryTask extends SystemTimer.SystemTimerTask {
     } 
     catch (CancelException ex) {
       // ignore
-
-      // @todo grid: do we need to deal with pool exceptions here?
-     } catch (BridgeWriterException ex) {
-       // Some exceptions from the bridge writer should not be logged.
-       Throwable cause = ex.getCause();
-       // BridgeWriterExceptions from the server are wrapped in CacheWriterExceptions
-       if (cause != null && cause instanceof CacheWriterException)
-           cause = cause.getCause();
-       if (cause instanceof RegionDestroyedException ||
-           cause instanceof EntryNotFoundException ||
-           cause instanceof CancelException) {
-         if (logger.isDebugEnabled()) {
-           logger.debug("Exception in expiration task", ex);
-         }
-       } else {
-         logger.fatal(LocalizedMessage.create(LocalizedStrings.ExpiryTask_EXCEPTION_IN_EXPIRATION_TASK), ex);
-       }
     } 
      catch (VirtualMachineError err) {
        SystemFailure.initiateFailure(err);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/FindDurableQueueProcessor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/FindDurableQueueProcessor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/FindDurableQueueProcessor.java
index 1366f94..8d782a9 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/FindDurableQueueProcessor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/FindDurableQueueProcessor.java
@@ -76,11 +76,11 @@ public class FindDurableQueueProcessor extends ReplyProcessor21 {
   private static void findLocalDurableQueues(ClientProxyMembershipID proxyId, ArrayList<ServerLocation> matches) {
     Cache c = GemFireCacheImpl.getInstance();
     if(c!=null) {
-      List l = c.getBridgeServers();
+      List l = c.getCacheServers();
       if(l!=null) {
         Iterator i = l.iterator();
         while(i.hasNext()) {
-          BridgeServerImpl bs = (BridgeServerImpl)i.next();
+          CacheServerImpl bs = (CacheServerImpl)i.next();
           if(bs.getAcceptor().getCacheClientNotifier().getClientProxy(proxyId)!=null) {
             ServerLocation loc = new ServerLocation(bs.getExternalAddress(),bs.getPort());
             matches.add(loc);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index 79bcbc2..4bf0f42 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -136,7 +136,6 @@ import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.cache.query.internal.cq.CqServiceProvider;
 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.util.ObjectSizer;
 import com.gemstone.gemfire.cache.wan.GatewayReceiver;
@@ -371,11 +370,11 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
   private volatile DistributionAdvisee sqlfAdvisee;
 
   /**
-   * the list of all bridge servers. CopyOnWriteArrayList is used to allow concurrent add, remove and retrieval
-   * operations. It is assumed that the traversal operations on bridge servers list vastly outnumber the mutative
+   * the list of all cache servers. CopyOnWriteArrayList is used to allow concurrent add, remove and retrieval
+   * operations. It is assumed that the traversal operations on cache servers list vastly outnumber the mutative
    * operations such as add, remove.
    */
-  private volatile List allBridgeServers = new CopyOnWriteArrayList();
+  private volatile List allCacheServers = new CopyOnWriteArrayList();
 
   /**
    * Controls updates to the list of all gateway senders
@@ -664,7 +663,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     sb.append("; lockLease = " + this.lockLease);
     sb.append("; lockTimeout = " + this.lockTimeout);
     // sb.append("; rootRegions = (" + this.rootRegions + ")");
-    // sb.append("; bridgeServers = (" + this.bridgeServers + ")");
+    // sb.append("; cacheServers = (" + this.cacheServers + ")");
     // sb.append("; regionAttributes = (" + this.listRegionAttributes());
     // sb.append("; gatewayHub = " + gatewayHub);
     if (this.creationStack != null) {
@@ -1513,7 +1512,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
   }
 
   /**
-   * Close the distributed system, bridge servers, and gateways. Clears the rootRegions and partitionedRegions map.
+   * Close the distributed system, cache servers, and gateways. Clears the rootRegions and partitionedRegions map.
    * Marks the cache as closed.
    *
    * @see SystemFailure#emergencyClose()
@@ -1546,14 +1545,14 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     inst.disconnectCause = SystemFailure.getFailure();
     inst.isClosing = true;
 
-    // Clear bridge servers
+    // Clear cache servers
     if (DEBUG) {
-      System.err.println("DEBUG: Close bridge servers");
+      System.err.println("DEBUG: Close cache servers");
     }
     {
-      Iterator allBridgeServersItr = inst.allBridgeServers.iterator();
-      while (allBridgeServersItr.hasNext()) {
-        BridgeServerImpl bs = (BridgeServerImpl) allBridgeServersItr.next();
+      Iterator allCacheServersItr = inst.allCacheServers.iterator();
+      while (allCacheServersItr.hasNext()) {
+        CacheServerImpl bs = (CacheServerImpl) allCacheServersItr.next();
         AcceptorImpl ai = bs.getAcceptor();
         if (ai != null) {
           ai.emergencyClose();
@@ -1986,7 +1985,6 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
       TXStateProxy tx = null;
       try {
         this.keepAlive = keepalive;
-        PoolManagerImpl.setKeepAlive(keepalive);
 
         if (this.txMgr != null) {
           tx = this.txMgr.internalSuspend();
@@ -2044,7 +2042,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
           stopRedisServer();
 
           // no need to track PR instances since we won't create any more
-          // bridgeServers or gatewayHubs
+          // cacheServers or gatewayHubs
           if (this.partitionedRegions != null) {
             if (isDebugEnabled) {
               logger.debug("{}: clearing partitioned regions...", this);
@@ -2616,12 +2614,12 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     final boolean isDebugEnabled = logger.isDebugEnabled();
     
     if (isDebugEnabled) {
-      logger.debug("{}: stopping bridge servers...", this);
+      logger.debug("{}: stopping cache servers...", this);
     }
-    boolean stoppedBridgeServer = false;
-    Iterator allBridgeServersIterator = this.allBridgeServers.iterator();
-    while (allBridgeServersIterator.hasNext()) {
-      BridgeServerImpl bridge = (BridgeServerImpl) allBridgeServersIterator.next();
+    boolean stoppedCacheServer = false;
+    Iterator allCacheServersIterator = this.allCacheServers.iterator();
+    while (allCacheServersIterator.hasNext()) {
+      CacheServerImpl bridge = (CacheServerImpl) allCacheServersIterator.next();
       if (isDebugEnabled) {
         logger.debug("stopping bridge {}", bridge);
       }
@@ -2632,11 +2630,11 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
           logger.debug("Ignored cache closure while closing bridge {}", bridge, e);
         }
       }
-      allBridgeServers.remove(bridge);
-      stoppedBridgeServer = true;
+      allCacheServers.remove(bridge);
+      stoppedCacheServer = true;
     }
-    if (stoppedBridgeServer) {
-      // now that all the bridge servers have stopped empty the static pool of commBuffers it might have used.
+    if (stoppedCacheServer) {
+      // now that all the cache servers have stopped empty the static pool of commBuffers it might have used.
       ServerConnection.emptyCommBufferPool();
     }
     
@@ -3784,10 +3782,6 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     return this.eventThreadPool;
   }
 
-  public BridgeServer addBridgeServer() {
-    return (BridgeServer) addCacheServer();
-  }
-
   public CacheServer addCacheServer() {
     return addCacheServer(false);
   }
@@ -3798,8 +3792,8 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     }
     stopper.checkCancelInProgress(null);
 
-    BridgeServerImpl bridge = new BridgeServerImpl(this, isGatewayReceiver);
-    allBridgeServers.add(bridge);
+    CacheServerImpl bridge = new CacheServerImpl(this, isGatewayReceiver);
+    allCacheServers.add(bridge);
 
     sendAddCacheServerProfileMessage();
     return bridge;
@@ -3972,33 +3966,29 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     }
   }
 
-  public List getBridgeServers() {
-    return getCacheServers();
-  }
-
   public List getCacheServers() {
-    List bridgeServersWithoutReceiver = null;
-    if (!allBridgeServers.isEmpty()) {
-    Iterator allBridgeServersIterator = allBridgeServers.iterator();
-    while (allBridgeServersIterator.hasNext()) {
-      BridgeServerImpl bridgeServer = (BridgeServerImpl) allBridgeServersIterator.next();
-      // If BridgeServer is a GatewayReceiver, don't return as part of CacheServers
-      if (!bridgeServer.isGatewayReceiver()) {
-        if (bridgeServersWithoutReceiver == null) {
-          bridgeServersWithoutReceiver = new ArrayList();
+    List cacheServersWithoutReceiver = null;
+    if (!allCacheServers.isEmpty()) {
+    Iterator allCacheServersIterator = allCacheServers.iterator();
+    while (allCacheServersIterator.hasNext()) {
+      CacheServerImpl cacheServer = (CacheServerImpl) allCacheServersIterator.next();
+      // If CacheServer is a GatewayReceiver, don't return as part of CacheServers
+      if (!cacheServer.isGatewayReceiver()) {
+        if (cacheServersWithoutReceiver == null) {
+          cacheServersWithoutReceiver = new ArrayList();
         }
-        bridgeServersWithoutReceiver.add(bridgeServer);
+        cacheServersWithoutReceiver.add(cacheServer);
       }
     }
     }
-    if (bridgeServersWithoutReceiver == null) {
-      bridgeServersWithoutReceiver = Collections.emptyList();
+    if (cacheServersWithoutReceiver == null) {
+      cacheServersWithoutReceiver = Collections.emptyList();
     }
-    return bridgeServersWithoutReceiver;
+    return cacheServersWithoutReceiver;
   }
 
-  public List getBridgeServersAndGatewayReceiver() {
-    return allBridgeServers;
+  public List getCacheServersAndGatewayReceiver() {
+    return allCacheServers;
   }
 
   /**
@@ -4126,9 +4116,9 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
       boolean hasSerialSenders = hasSerialSenders(r);
       boolean result = hasSerialSenders;
       if (!result) {
-        Iterator allBridgeServersIterator = allBridgeServers.iterator();
-        while (allBridgeServersIterator.hasNext()) {
-          BridgeServerImpl server = (BridgeServerImpl) allBridgeServersIterator.next();
+        Iterator allCacheServersIterator = allCacheServers.iterator();
+        while (allCacheServersIterator.hasNext()) {
+          CacheServerImpl server = (CacheServerImpl) allCacheServersIterator.next();
           if (!server.getNotifyBySubscription()) {
             result = true;
             break;
@@ -4182,7 +4172,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     stopper.checkCancelInProgress(null);
 
     if (!this.isServer) {
-      return (this.allBridgeServers.size() > 0);
+      return (this.allCacheServers.size() > 0);
     } else {
       return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java
index acafd6d..13d6068 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java
@@ -45,7 +45,7 @@ public abstract class GridAdvisor extends DistributionAdvisor {
 
   private static final Filter BRIDGE_SERVER_FILTER = new Filter() {
       public boolean include(Profile profile) {
-        return profile instanceof BridgeServerAdvisor.BridgeServerProfile;
+        return profile instanceof CacheServerAdvisor.CacheServerProfile;
       }
     };
   
@@ -327,9 +327,9 @@ public abstract class GridAdvisor extends DistributionAdvisor {
         boolean exchangeProfiles, final List<Profile> replyProfiles) {
       final GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
       if (cache != null && !cache.isClosed()) {
-        List<?> bridgeServers = cache.getBridgeServersAndGatewayReceiver();
+        List<?> bridgeServers = cache.getCacheServersAndGatewayReceiver();
         for (int i = 0; i < bridgeServers.size(); i++) {
-          BridgeServerImpl bsi = (BridgeServerImpl)bridgeServers.get(i);
+          CacheServerImpl bsi = (CacheServerImpl)bridgeServers.get(i);
           if (bsi.isRunning()) {
             if(bsi.getProfile().equals(this)) {
               continue;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index 7c1ec89..9e5bcd2 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -100,7 +100,6 @@ import com.gemstone.gemfire.cache.TransactionId;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.ServerOperationException;
 import com.gemstone.gemfire.cache.client.SubscriptionNotEnabledException;
-import com.gemstone.gemfire.cache.client.internal.BridgePoolImpl;
 import com.gemstone.gemfire.cache.client.internal.Connection;
 import com.gemstone.gemfire.cache.client.internal.Endpoint;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
@@ -134,7 +133,6 @@ import com.gemstone.gemfire.cache.query.internal.index.IndexCreationData;
 import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
 import com.gemstone.gemfire.cache.query.internal.index.IndexProtocol;
 import com.gemstone.gemfire.cache.query.internal.index.IndexUtils;
-import com.gemstone.gemfire.cache.util.BridgeWriterException;
 import com.gemstone.gemfire.cache.util.ObjectSizer;
 import com.gemstone.gemfire.cache.wan.GatewaySender;
 import com.gemstone.gemfire.distributed.DistributedMember;
@@ -674,9 +672,7 @@ public class LocalRegion extends AbstractRegion
     }
     
     // initialize client to server proxy
-    this.srp = ((this.getPoolName() != null)
-                || isBridgeLoader(this.getCacheLoader())
-                || isBridgeWriter(this.getCacheWriter()))
+    this.srp = (this.getPoolName() != null)
       ? new ServerRegionProxy(this)
       : null;
     this.imageState =
@@ -3986,22 +3982,6 @@ public class LocalRegion extends AbstractRegion
     reinitialize(inputStream, event);
   }
 
-//   public void createRegionOnServer() throws CacheWriterException
-//   {
-//     if (basicGetWriter() instanceof BridgeWriter) {
-//       if (getParentRegion() != null) {
-//         BridgeWriter bw = (BridgeWriter)basicGetWriter();
-//         bw.createRegionOnServer(getParentRegion().getFullPath(), getName());
-//       }
-//       else {
-//        throw new CacheWriterException(LocalizedStrings.LocalRegion_REGION_0_IS_A_ROOT_REGION_ONLY_NONROOT_REGIONS_CAN_BE_CREATED_ON_THE_SERVER.toLocalizedString(getFullPath()));
-//       }
-//     }
-//     else {
-//      throw new CacheWriterException(LocalizedStrings.LocalRegion_SERVER_REGION_CREATION_IS_ONLY_SUPPORTED_ON_CLIENT_SERVER_TOPOLOGIES_THE_CURRENT_CACHEWRITER_IS_0.toLocalizedString(this.cacheWriter));
-//     }
-//   }
-
   public void registerInterest(Object key)
   {
     registerInterest(key, false);
@@ -4068,13 +4048,8 @@ public class LocalRegion extends AbstractRegion
       throw new IllegalStateException(LocalizedStrings.LocalRegion_DURABLE_FLAG_ONLY_APPLICABLE_FOR_DURABLE_CLIENTS.toLocalizedString());
     }
     if (!proxy.getPool().getSubscriptionEnabled()) {
-      if (proxy.getPool() instanceof BridgePoolImpl) {
-        String msg = "Interest registration requires establishCallbackConnection to be set to true.";
-        throw new BridgeWriterException(msg);
-      } else {
-        String msg = "Interest registration requires a pool whose queue is enabled.";
-        throw new SubscriptionNotEnabledException(msg);
-      }
+      String msg = "Interest registration requires a pool whose queue is enabled.";
+      throw new SubscriptionNotEnabledException(msg);
     }
 
     if (getAttributes().getDataPolicy().withReplication() // fix for bug 36185
@@ -4101,7 +4076,7 @@ public class LocalRegion extends AbstractRegion
       this.clearKeysOfInterest(key, interestType, pol);
       // Checking for the Dunit test(testRegisterInterst_Destroy_Concurrent) flag
       if (PoolImpl.BEFORE_REGISTER_CALLBACK_FLAG) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.beforeInterestRegistration();
       }// Test Code Ends
       final byte regionDataPolicy = getAttributes().getDataPolicy().ordinal;
@@ -9786,7 +9761,7 @@ public class LocalRegion extends AbstractRegion
       }
     }
 
-    RegionEventImpl event = new BridgeRegionEventImpl(this, Operation.REGION_DESTROY,
+    RegionEventImpl event = new ClientRegionEventImpl(this, Operation.REGION_DESTROY,
          callbackArg,false, client.getDistributedMember(), client/* context */, eventId);
 
     basicDestroyRegion(event, true);
@@ -9811,7 +9786,7 @@ public class LocalRegion extends AbstractRegion
       }
     }
 
-    RegionEventImpl event = new BridgeRegionEventImpl(this, Operation.REGION_CLEAR,
+    RegionEventImpl event = new ClientRegionEventImpl(this, Operation.REGION_CLEAR,
          callbackArg,false, client.getDistributedMember(), client/* context */, eventId);
 
     basicClear(event, true);
@@ -11310,7 +11285,7 @@ public class LocalRegion extends AbstractRegion
    */
   protected boolean shouldNotifyBridgeClients()
   {
-    return (this.cache.getBridgeServers().size() > 0)
+    return (this.cache.getCacheServers().size() > 0)
         && !this.isUsedForPartitionedRegionAdmin
         && !this.isUsedForPartitionedRegionBucket
         && !this.isUsedForMetaRegion;
@@ -11444,7 +11419,7 @@ public class LocalRegion extends AbstractRegion
       predicate = predicate.trim();
 
       // Compare the query patterns to the 'predicate'. If one matches,
-      // send it as is to the BridgeLoader
+      // send it as is to the server
       boolean matches = false;
       for (int i=0; i<QUERY_PATTERNS.length; i++) {
         if (QUERY_PATTERNS[i].matcher(predicate).matches()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
index c007891..629c5a4 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
@@ -5457,9 +5457,6 @@ public class PartitionedRegion extends LocalRegion implements
   protected void cacheWriterChanged(CacheWriter p_oldWriter) {
     CacheWriter oldWriter = p_oldWriter;
     super.cacheWriterChanged(oldWriter);
-    if (isBridgeWriter(oldWriter)) {
-      oldWriter = null;
-    }
     if (oldWriter == null ^ basicGetWriter() == null) {
       new UpdateAttributesProcessor(this).distribute();
     }
@@ -5469,9 +5466,6 @@ public class PartitionedRegion extends LocalRegion implements
   @Override
   protected void cacheLoaderChanged(CacheLoader oldLoader) {
     CacheLoader myOldLoader = oldLoader;
-    if (isBridgeLoader(oldLoader)) {
-      myOldLoader = null;
-    }
     this.dataStore.cacheLoaderChanged(basicGetLoader(), myOldLoader);
     super.cacheLoaderChanged(oldLoader);
     if (myOldLoader == null ^ basicGetLoader() == null) {
@@ -5902,7 +5896,7 @@ public class PartitionedRegion extends LocalRegion implements
     Collections.addAll(localServerGroups, MemberAttributes.parseGroups(null, c.getSystem().getConfig().getGroups()));
     
     for (Object object : servers) {
-      BridgeServerImpl server = (BridgeServerImpl)object;
+      CacheServerImpl server = (CacheServerImpl)object;
       if (server.isRunning() && (server.getExternalAddress() != null)) {
         Collections.addAll(localServerGroups, server.getGroups());
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolFactoryImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolFactoryImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolFactoryImpl.java
index 4569184..0838d29 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolFactoryImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolFactoryImpl.java
@@ -27,8 +27,6 @@ import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.RegionService;
 import com.gemstone.gemfire.cache.client.Pool;
 import com.gemstone.gemfire.cache.client.PoolFactory;
-import com.gemstone.gemfire.cache.client.internal.BridgePoolImpl;
-import com.gemstone.gemfire.cache.client.internal.BridgePoolImpl.LBPolicy;
 import com.gemstone.gemfire.cache.client.internal.LocatorDiscoveryCallback;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.query.QueryService;
@@ -286,141 +284,14 @@ public class PoolFactoryImpl implements PoolFactory {
     this.attributes.servers.addAll(cp.getServers());
   }
 
-  public void init(Properties properties, boolean usedByBridgeWriter,
-      boolean usedByGateway, GatewaySender sender) {
+  public void init(GatewaySender sender) {
+    this.attributes.setGateway(true);
     this.attributes.setGatewaySender(sender);
-    init(properties, usedByBridgeWriter, usedByGateway);
-  }
-  /**
-   * Used to create a pool given the old Bridge properties
-   * @param properties from a BridgeWriter or BridgeLoader
-   * @param usedByBridgeWriter true  if props from BridgeWriter;
-   *                           false if props from BridgeLoader.
-   * *param usedByGateway true if props from GatewayImpl.
-   * @since 5.7
-   */
-  public void init(Properties properties, boolean usedByBridgeWriter,
-                   boolean usedByGateway) {
-    this.attributes.setBridge(usedByBridgeWriter || !usedByGateway);
-    this.attributes.setBridgeWriter(usedByBridgeWriter);
-    this.attributes.setGateway(usedByGateway);
     setIdleTimeout(-1); // never time out
     setLoadConditioningInterval(-1); // never time out
     setMaxConnections(-1);
-    int endpointCount = 0;
-    boolean useLocators = false;
-    boolean useEndPoints = false;
-    IllegalArgumentException exception = null;
-    if (properties.containsKey(DistributionConfig.LOCATORS_NAME)) {
-      String locatorObject = properties
-          .getProperty(DistributionConfig.LOCATORS_NAME);
-      if (locatorObject != null && !locatorObject.equals("")) {
-        StringTokenizer locatorsOnThisVM = new StringTokenizer(locatorObject, ",");
-        while (locatorsOnThisVM.hasMoreTokens()) {
-          String localLocator = locatorsOnThisVM.nextToken();
-          DistributionLocatorId locatorId = new DistributionLocatorId(
-              localLocator);
-          addLocator(locatorId.getHost().getHostName(), locatorId.getPort());
-        }
-        useLocators = true;
-      }
-    }
-    if (!useLocators && properties.containsKey("endpoints")) {
-      useEndPoints = true;
-      String pv = properties.getProperty("endpoints");
-      StringTokenizer tokenizer = new StringTokenizer(pv, ",");
-      while (tokenizer.hasMoreTokens()) {
-        String serverdetail = tokenizer.nextToken();
-        int cIndex = serverdetail.indexOf("=");
-        // note we throw the name away
-//         String name = serverdetail.substring(0, cIndex);
-//         if (name != null) {
-//           name = name.trim();
-//         }
-        String remainder = serverdetail.substring(cIndex + 1);
-        cIndex = remainder.lastIndexOf(":");
-        String host = remainder.substring(0, cIndex);
-        if (host != null) {
-          host = host.trim();
-        }
-        String port = remainder.substring(cIndex + 1);
-        if (port != null) {
-          port = port.trim();
-        }
-        try {
-          addServer(host, Integer.parseInt(port));
-          endpointCount++;
-        } catch (IllegalArgumentException e) {
-          if (!(e.getCause() instanceof UnknownHostException)) {
-            throw e;
-          } else {
-            exception = e;
-          }
-        }
-      }
-      if ((endpointCount == 0) && (exception != null)) {
-        IllegalArgumentException ex = new IllegalArgumentException("Couldn't find any Endpoint. " + exception.getMessage());
-        ex.initCause(exception.getCause());
-        throw ex;
-      }
-    }
-    if(!useLocators && !useEndPoints) {
-      throw new IllegalArgumentException(
-          "Property 'locators ' or 'endpoints' must be specified");
-    }
-    // @todo grid: handshakeTimeout ignored
-    {
-      // @todo grid: roundRobin and appAssisted ignored
-      LBPolicy policy = new LBPolicy(properties.getProperty("LBPolicy",
-          LBPolicy.STICKY_PROPERTY_NAME));
-      setThreadLocalConnections(policy.isSticky());
-    }
-    
-    if (properties.containsKey("retryAttempts")) {
-      String strRetryAttempts = properties.getProperty("retryAttempts");
-      setRetryAttempts(Integer.parseInt(strRetryAttempts));
-    }
-    if (properties.containsKey("retryInterval")) {
-      String strRetryAttempts = properties.getProperty("retryInterval");
-      setPingInterval(Integer.parseInt(strRetryAttempts));
-    }
-    if (properties.containsKey("establishCallbackConnection")) {
-      String str = properties.getProperty("establishCallbackConnection");
-      setSubscriptionEnabled(Boolean.valueOf(str).booleanValue());
-    }
-    if (properties.containsKey("enablePRSingleHop")) {
-      String str = properties.getProperty("enablePRSingleHop");
-      setPRSingleHopEnabled(Boolean.valueOf(str).booleanValue());
-    }
-    if (properties.containsKey("connectionsPerServer")) {
-      String str = properties.getProperty("connectionsPerServer");
-      setMinConnections(Integer.parseInt(str)*endpointCount);
-    } else {
-      setMinConnections(1*endpointCount);
-    }
-    if (properties.containsKey("redundancyLevel")) {
-      String str = properties.getProperty("redundancyLevel");
-      setSubscriptionRedundancy(Integer.parseInt(str));
-    }
-    if (properties.containsKey("readTimeout")) {
-      String strReadTimeout = properties.getProperty("readTimeout");
-      setReadTimeout(Integer.parseInt(strReadTimeout));
-    }
-    if (properties.containsKey("socketBufferSize")) {
-      String strSocketBufferSize = properties.getProperty("socketBufferSize");
-      setSocketBufferSize(Integer.parseInt(strSocketBufferSize));
-    }
-    if (properties.containsKey("messageTrackingTimeout")) {
-      String pv = properties.getProperty("messageTrackingTimeout");
-      setSubscriptionMessageTrackingTimeout(Integer.parseInt(pv));
-    }
-    if(properties.containsKey("clientAckInterval") ) {
-      String pv = properties.getProperty("clientAckInterval");
-      setSubscriptionAckInterval(Integer.parseInt(pv));
-    }
-    if(usedByGateway && exception!= null) {
-      throw exception;
-    }
+    setMinConnections(0);
+    setThreadLocalConnections(true);
   }
   
   /**
@@ -441,11 +312,7 @@ public class PoolFactoryImpl implements PoolFactory {
         registry.creatingPool();
       }
     }
-    if (this.attributes.isBridge()) {
-      return new BridgePoolImpl(this.pm, name, this.attributes);
-    } else {
-      return PoolImpl.create(this.pm, name, this.attributes);
-    }
+    return PoolImpl.create(this.pm, name, this.attributes);
   }
 
   /**
@@ -487,19 +354,7 @@ public class PoolFactoryImpl implements PoolFactory {
     public transient LocatorDiscoveryCallback locatorCallback = null; //only used by tests
     public GatewaySender gatewaySender = null;
     /**
-     * True if this factory needs to produce a pool for use by BridgeWriter
-     * or BridgeLoader.
-     */
-    public boolean bridge = false;
-    /**
-     * True if bridge is true and the pool is used by a BridgeWriter.
-     * False if bridge is true and the pool is used by a BridgeLoader.
-     * Ignore this attribute if bridge is false.
-     */
-    public boolean bridgeWriter = false;
-    /**
-     * True if bridge is true and the pool is used by a Gateway.
-     * Ignore this attribute if bridge is false.
+     * True if the pool is used by a Gateway.
      */
     public boolean gateway = false;
 
@@ -554,18 +409,6 @@ public class PoolFactoryImpl implements PoolFactory {
     public String getServerGroup() {
       return this.serverGroup;
     }
-    public boolean isBridge() {
-      return this.bridge;
-    }
-    public void setBridge(boolean v) {
-      this.bridge = v;
-    }
-    public boolean isBridgeWriter() {
-      return this.bridgeWriter;
-    }
-    public void setBridgeWriter(boolean v) {
-      this.bridgeWriter = v;
-    }
     public boolean isGateway() {
       return this.gateway;
     }
@@ -648,7 +491,6 @@ public class PoolFactoryImpl implements PoolFactory {
       DataSerializer.writeString(this.serverGroup, out);
       DataSerializer.writeArrayList(this.locators, out);
       DataSerializer.writeArrayList(this.servers, out);
-      DataSerializer.writePrimitiveBoolean(this.bridge, out);
       DataSerializer.writePrimitiveInt(this.statisticInterval, out);
       DataSerializer.writePrimitiveBoolean(this.multiuserSecureModeEnabled,out);
     }
@@ -671,7 +513,6 @@ public class PoolFactoryImpl implements PoolFactory {
       this.serverGroup = DataSerializer.readString(in);
       this.locators = DataSerializer.readArrayList(in);
       this.servers = DataSerializer.readArrayList(in);
-      this.bridge = DataSerializer.readPrimitiveBoolean(in);
       this.statisticInterval= DataSerializer.readPrimitiveInt(in);
       this.multiuserSecureModeEnabled = DataSerializer.readPrimitiveBoolean(in);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolManagerImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolManagerImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolManagerImpl.java
index 289a3f5..25ba55d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolManagerImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/PoolManagerImpl.java
@@ -22,7 +22,6 @@ import com.gemstone.gemfire.cache.Region;
 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.client.internal.RegisterDataSerializersOp;
 import com.gemstone.gemfire.cache.client.internal.RegisterInstantiatorsOp;
@@ -102,24 +101,6 @@ public class PoolManagerImpl {
   }
   
   /**
-   * Set the keep alive flag before closing. Only for use with the deprecated
-   * BridgeWriter/Loader code. A BridgeWriter is automatically
-   * closed then the last region is disconnected from it,
-   * so we need to mark the connections as keep alive
-   * before we close the regions that use the bridge writer/loader
-   * 
-   * @param keepAlive
-   */
-  public static void setKeepAlive(boolean keepAlive) {
-    for(Iterator<Pool> itr = PoolManager.getAll().values().iterator(); itr.hasNext(); ) {
-      Pool nextPool = itr.next();
-      if(nextPool instanceof BridgePoolImpl) {
-        BridgePoolImpl bridgePool = (BridgePoolImpl) nextPool;
-        bridgePool.setKeepAlive(keepAlive);
-      }
-    }
-  }
-  /**
    * Destroys all created pool in this manager.
    */
   public void close(boolean keepAlive) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEventImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEventImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEventImpl.java
index 7d96986..08a6bad 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEventImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEventImpl.java
@@ -282,7 +282,7 @@ public class RegionEventImpl
   }
   
   public ClientProxyMembershipID getContext() {
-    // regular region events do not have bridge context - see BridgeRegionEventImpl
+    // regular region events do not have a context - see ClientRegionEventImpl
     return null;
   }
   



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

Posted by bs...@apache.org.
GEODE-243: remove deprecated Bridge feature

All the public classes and most of the internal ones
have been removed. The cache xsd also has its bridge
elements removed.
The deprecated admin package did not have its bridge
apis removed since the entire admin package is going
to be removed as a seperate project.
The external javadocs have also been updated to no
longer use the term "bridge".
Localized strings were changed to no longer use "bridge".
Internal code and test code continues to have some
bridge references. Additional cleanup will be done
in the future.


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

Branch: refs/heads/feature/GEODE-77
Commit: 2eb4e17500a8faad994d80c08a149b1ee9baaa3e
Parents: 7111906
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Aug 27 11:02:27 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Oct 6 10:47:10 2015 -0700

----------------------------------------------------------------------
 .../gemfire/admin/GemFireMemberStatus.java      |   13 +-
 .../gemfire/admin/SystemMemberBridgeServer.java |    5 +-
 .../gemfire/admin/SystemMemberCache.java        |   27 -
 .../DistributedSystemHealthMonitor.java         |    2 +-
 .../admin/internal/SystemMemberCacheImpl.java   |   17 +-
 .../jmx/internal/SystemMemberCacheJmxImpl.java  |   13 -
 .../gemfire/cache/AttributesFactory.java        |   31 -
 .../java/com/gemstone/gemfire/cache/Cache.java  |   27 -
 .../gemfire/cache/DynamicRegionFactory.java     |  121 +-
 .../gemfire/cache/client/PoolFactory.java       |   10 +-
 .../cache/client/internal/AbstractOp.java       |    2 +-
 .../cache/client/internal/BridgePoolImpl.java   |  479 -----
 .../internal/BridgeServerLoadMessage.java       |   99 --
 .../client/internal/CacheServerLoadMessage.java |   99 ++
 .../cache/client/internal/DestroyOp.java        |    5 -
 .../client/internal/EndpointManagerImpl.java    |    8 +-
 .../internal/ExplicitConnectionSourceImpl.java  |   85 +-
 .../gemfire/cache/client/internal/GetOp.java    |    5 -
 .../gemfire/cache/client/internal/PoolImpl.java |   14 +-
 .../gemfire/cache/client/internal/PutOp.java    |    5 -
 .../cache/client/internal/QueueManagerImpl.java |   14 +-
 .../cache/client/internal/QueueStateImpl.java   |    6 +-
 .../internal/RegisterDataSerializersOp.java     |    8 +-
 .../internal/RegisterInstantiatorsOp.java       |    8 +-
 .../client/internal/ServerRegionProxy.java      |   44 +-
 .../internal/FunctionServiceManager.java        |    3 +-
 .../gemfire/cache/server/ServerLoadProbe.java   |   10 +-
 .../gemfire/cache/server/ServerMetrics.java     |   10 +-
 .../cache/server/internal/LoadMonitor.java      |   12 +-
 .../gemfire/cache/util/BridgeClient.java        |  156 --
 .../gemfire/cache/util/BridgeLoader.java        |  607 -------
 .../gemfire/cache/util/BridgeMembership.java    |   55 -
 .../cache/util/BridgeMembershipEvent.java       |   23 -
 .../cache/util/BridgeMembershipListener.java    |   41 -
 .../util/BridgeMembershipListenerAdapter.java   |   43 -
 .../gemfire/cache/util/BridgeServer.java        |  442 -----
 .../gemfire/cache/util/BridgeWriter.java        |  795 ---------
 .../cache/util/BridgeWriterException.java       |   48 -
 .../util/EndpointDoesNotExistException.java     |   34 -
 .../gemfire/cache/util/EndpointException.java   |   61 -
 .../cache/util/EndpointExistsException.java     |   28 -
 .../cache/util/EndpointInUseException.java      |   27 -
 .../util/IncompatibleVersionException.java      |   47 -
 .../util/ServerRefusedConnectionException.java  |   36 -
 .../UniversalMembershipListenerAdapter.java     |  352 ----
 .../cache/util/UnknownVersionException.java     |   39 -
 .../gemfire/cache/util/VersionException.java    |   45 -
 .../gemstone/gemfire/cache/util/package.html    |    3 -
 .../gemfire/distributed/ServerLauncher.java     |    4 +-
 .../internal/InternalDistributedSystem.java     |   12 +-
 .../distributed/internal/ServerLocator.java     |   10 +-
 .../jgroup/JGroupMembershipManager.java         |    6 +-
 .../gemstone/gemfire/internal/DSFIDFactory.java |   12 +-
 .../internal/DataSerializableFixedID.java       |    6 +-
 .../internal/admin/ClientStatsManager.java      |    7 -
 .../gemfire/internal/admin/GemFireVM.java       |    2 +-
 .../admin/remote/BridgeServerResponse.java      |   18 +-
 .../admin/remote/DurableClientInfoResponse.java |    4 +-
 .../admin/remote/RemoteBridgeServer.java        |   10 +-
 .../internal/admin/remote/RemoteCacheInfo.java  |    6 +-
 .../internal/admin/remote/RemoteGemFireVM.java  |    2 +-
 .../internal/cache/AbstractBridgeServer.java    |  425 -----
 .../internal/cache/AbstractCacheServer.java     |  398 +++++
 .../gemfire/internal/cache/AbstractRegion.java  |   52 -
 .../gemfire/internal/cache/BridgeObserver.java  |   89 -
 .../internal/cache/BridgeObserverAdapter.java   |  107 --
 .../internal/cache/BridgeObserverHolder.java    |   53 -
 .../internal/cache/BridgeRegionEventImpl.java   |  108 --
 .../internal/cache/BridgeServerAdvisor.java     |  165 --
 .../internal/cache/BridgeServerImpl.java        |  816 ---------
 .../gemfire/internal/cache/BucketAdvisor.java   |    2 +-
 .../gemfire/internal/cache/CacheConfig.java     |    8 +-
 .../internal/cache/CacheServerAdvisor.java      |  164 ++
 .../gemfire/internal/cache/CacheServerImpl.java |  812 +++++++++
 .../internal/cache/ClientRegionEventImpl.java   |  108 ++
 .../internal/cache/ClientServerObserver.java    |   90 +
 .../cache/ClientServerObserverAdapter.java      |  107 ++
 .../cache/ClientServerObserverHolder.java       |   53 +
 .../internal/cache/DestroyRegionOperation.java  |    6 +-
 .../cache/DistributedClearOperation.java        |    6 +-
 .../internal/cache/DistributedRegion.java       |    6 -
 .../gemfire/internal/cache/ExpiryTask.java      |   18 -
 .../cache/FindDurableQueueProcessor.java        |    4 +-
 .../internal/cache/GemFireCacheImpl.java        |   92 +-
 .../gemfire/internal/cache/GridAdvisor.java     |    6 +-
 .../gemfire/internal/cache/LocalRegion.java     |   41 +-
 .../internal/cache/PartitionedRegion.java       |    8 +-
 .../gemfire/internal/cache/PoolFactoryImpl.java |  171 +-
 .../gemfire/internal/cache/PoolManagerImpl.java |   19 -
 .../gemfire/internal/cache/RegionEventImpl.java |    2 +-
 .../internal/cache/doc-files/properties.html    |    4 +-
 .../cache/execute/InternalFunctionService.java  |    4 +-
 .../internal/cache/ha/HARegionQueue.java        |    6 +-
 .../PartitionedRegionObserverHolder.java        |    2 +-
 .../internal/cache/tier/ConnectionProxy.java    |  160 +-
 .../cache/tier/InternalBridgeMembership.java    |  715 --------
 .../cache/tier/InternalClientMembership.java    |  617 +++++++
 .../cache/tier/sockets/AcceptorImpl.java        |   20 +-
 .../cache/tier/sockets/CacheClientNotifier.java |   16 +-
 .../cache/tier/sockets/CacheClientProxy.java    |    6 +-
 .../cache/tier/sockets/CacheClientUpdater.java  |   12 +-
 .../tier/sockets/ClientBlacklistProcessor.java  |    4 +-
 .../RemoveClientFromBlacklistMessage.java       |    4 +-
 .../cache/tier/sockets/ServerConnection.java    |    8 +-
 .../doc-files/communication-architecture.fig    |    4 +-
 .../cache/xmlcache/BridgeServerCreation.java    |  249 ---
 .../internal/cache/xmlcache/CacheCreation.java  |   47 +-
 .../cache/xmlcache/CacheServerCreation.java     |  238 +++
 .../cache/xmlcache/CacheXmlGenerator.java       |   41 +-
 .../internal/cache/xmlcache/CacheXmlParser.java |   18 +-
 .../cache/xmlcache/ClientCacheCreation.java     |    5 -
 .../xmlcache/RegionAttributesCreation.java      |    9 -
 .../gemfire/internal/i18n/LocalizedStrings.java |   20 +-
 .../internal/i18n/ParentLocalizedStrings.java   |   81 +-
 .../internal/beans/CacheServerBridge.java       |   18 +-
 .../beans/GatewayReceiverMBeanBridge.java       |    4 +-
 .../internal/beans/ManagementAdapter.java       |   42 +-
 .../management/internal/beans/ServerBridge.java |    6 +-
 .../cli/functions/ContunuousQueryFunction.java  |    4 +-
 .../functions/GetMemberInformationFunction.java |    4 +-
 .../management/membership/ClientMembership.java |    8 +-
 .../gemfire/cache/cache-9.0.xsd                 |   26 -
 .../batterytest/greplogs/ExpectedStrings.java   |    3 +-
 .../cache/AttributesFactoryJUnitTest.java       |   18 -
 .../gemstone/gemfire/cache/ClientHelper.java    |    1 -
 .../cache/ConnectionPoolAndLoaderDUnitTest.java |    4 +-
 .../internal/AutoConnectionSourceDUnitTest.java |   23 +-
 .../AutoConnectionSourceWithUDPDUnitTest.java   |    4 +-
 .../internal/ConnectionPoolImplJUnitTest.java   |   11 +-
 .../internal/LocatorLoadBalancingDUnitTest.java |   10 +-
 .../cache/client/internal/LocatorTestBase.java  |   12 +-
 .../management/MemoryThresholdsDUnitTest.java   |    4 +-
 .../MemoryThresholdsOffHeapDUnitTest.java       |    4 +-
 .../query/dunit/PdxStringQueryDUnitTest.java    |   16 +-
 .../query/dunit/QueryUsingPoolDUnitTest.java    |   12 +-
 .../cache/query/dunit/RemoteQueryDUnitTest.java |   90 +-
 ...esourceManagerWithQueryMonitorDUnitTest.java |    4 +-
 ...rrentIndexInitOnOverflowRegionDUnitTest.java |    4 +-
 .../cache30/BridgeMembershipDUnitTest.java      | 1642 ------------------
 .../BridgeMembershipSelectorDUnitTest.java      |   16 -
 .../gemfire/cache30/BridgeTestCase.java         |  376 ----
 .../gemfire/cache30/BridgeWriterDUnitTest.java  |  418 -----
 .../cache30/BridgeWriterSelectorDUnitTest.java  |   16 -
 .../gemfire/cache30/Bug38741DUnitTest.java      |   10 +-
 .../gemfire/cache30/CacheXml30DUnitTest.java    |  127 --
 .../gemfire/cache30/CacheXml40DUnitTest.java    |    8 +-
 .../gemfire/cache30/CacheXml41DUnitTest.java    |   28 +-
 .../gemfire/cache30/CacheXml45DUnitTest.java    |    4 +-
 .../gemfire/cache30/CacheXml51DUnitTest.java    |    8 +-
 .../gemfire/cache30/CacheXml57DUnitTest.java    |    1 -
 .../cache30/ClientMembershipDUnitTest.java      | 1642 ++++++++++++++++++
 .../ClientMembershipSelectorDUnitTest.java      |   16 +
 .../ClientRegisterInterestDUnitTest.java        |  418 +++++
 ...ClientRegisterInterestSelectorDUnitTest.java |   16 +
 .../gemfire/cache30/ClientServerTestCase.java   |  376 ++++
 .../AbstractServerLauncherJUnitTestCase.java    |    4 +-
 .../ServerLauncherLocalJUnitTest.java           |    4 +-
 .../ServerLauncherRemoteJUnitTest.java          |    8 +-
 .../internal/ProductUseLogDUnitTest.java        |    2 +-
 .../disttx/DistributedTransactionDUnitTest.java |    4 +-
 .../internal/cache/Bug39079DUnitTest.java       |    6 +-
 .../internal/cache/Bug41957DUnitTest.java       |    6 +-
 ...ssagesRegionCreationAndDestroyJUnitTest.java |   10 +-
 .../cache/ClientServerGetAllDUnitTest.java      |   40 +-
 .../cache/ClientServerTransactionDUnitTest.java |    6 +-
 .../cache/ConcurrentMapOpsDUnitTest.java        |    6 +-
 .../cache/DeltaPropagationDUnitTest.java        |    8 +-
 .../cache/DeltaPropagationStatsDUnitTest.java   |    2 +-
 .../internal/cache/DiskRegionJUnitTest.java     |    9 +-
 .../internal/cache/EventTrackerDUnitTest.java   |    8 +-
 .../internal/cache/GridAdvisorDUnitTest.java    |  106 +-
 .../internal/cache/HABug36773DUnitTest.java     |    8 +-
 .../HAOverflowMemObjectSizerDUnitTest.java      |   12 +-
 .../cache/execute/Bug51193DUnitTest.java        |    4 +-
 .../internal/cache/functions/TestFunction.java  |   10 +-
 .../cache/ha/Bug36853EventsExpiryDUnitTest.java |    8 +-
 .../internal/cache/ha/Bug48571DUnitTest.java    |    2 +-
 .../cache/ha/EventIdOptimizationDUnitTest.java  |   12 +-
 .../internal/cache/ha/FailoverDUnitTest.java    |   22 +-
 .../internal/cache/ha/HABugInPutDUnitTest.java  |    8 +-
 .../internal/cache/ha/HAClearDUnitTest.java     |   10 +-
 .../cache/ha/HAConflationDUnitTest.java         |    8 +-
 .../internal/cache/ha/HADuplicateDUnitTest.java |   10 +-
 .../cache/ha/HAEventIdPropagationDUnitTest.java |   10 +-
 .../internal/cache/ha/HAGIIDUnitTest.java       |   14 +-
 .../cache/ha/HARQueueNewImplDUnitTest.java      |   66 +-
 .../cache/ha/HASlowReceiverDUnitTest.java       |   10 +-
 .../ha/OperationsPropagationDUnitTest.java      |    8 +-
 .../internal/cache/ha/PutAllDUnitTest.java      |   12 +-
 .../cache/ha/StatsBugDUnitDisabledTest.java     |   14 +-
 .../cache/partitioned/Bug43684DUnitTest.java    |    4 +-
 .../fixed/FixedPartitioningTestBase.java        |    6 +-
 .../tier/sockets/AcceptorImplJUnitTest.java     |   62 +-
 ...rdCompatibilityCommandDUnitDisabledTest.java |  235 ---
 ...CompatibilityHandshakeDUnitDisabledTest.java |  218 ---
 ...mpatibilityHigherVersionClientDUnitTest.java |    4 +-
 ...rdCompatibilityMessageDUnitDisabledTest.java |  299 ----
 .../BridgeServerMaxConnectionsJUnitTest.java    |  221 ---
 ...geServerSelectorMaxConnectionsJUnitTest.java |   19 -
 .../tier/sockets/BridgeWriterMiscDUnitTest.java | 1384 ---------------
 .../BridgeWriterMiscSelectorDUnitTest.java      |   27 -
 .../cache/tier/sockets/Bug36269DUnitTest.java   |    4 +-
 .../cache/tier/sockets/Bug36457DUnitTest.java   |   18 +-
 .../cache/tier/sockets/Bug36805DUnitTest.java   |    6 +-
 .../cache/tier/sockets/Bug36995DUnitTest.java   |   11 +-
 .../cache/tier/sockets/Bug37210DUnitTest.java   |   10 +-
 .../CacheServerMaxConnectionsJUnitTest.java     |  220 +++
 ...heServerSelectorMaxConnectionsJUnitTest.java |   19 +
 .../cache/tier/sockets/CacheServerTestUtil.java |   51 +-
 .../CacheServerTransactionsDUnitTest.java       |    4 +-
 .../tier/sockets/ClearPropagationDUnitTest.java |    4 +-
 .../tier/sockets/ClientConflationDUnitTest.java |   20 +-
 .../sockets/ClientHealthMonitorJUnitTest.java   |    6 +-
 .../sockets/ClientInterestNotifyDUnitTest.java  |   14 +-
 .../tier/sockets/ClientServerMiscDUnitTest.java | 1381 +++++++++++++++
 .../ClientServerMiscSelectorDUnitTest.java      |   27 +
 .../cache/tier/sockets/ConflationDUnitTest.java |   24 +-
 .../tier/sockets/ConnectionProxyJUnitTest.java  |  257 +--
 .../DataSerializerPropogationDUnitTest.java     |   36 +-
 .../DestroyEntryPropagationDUnitTest.java       |   12 +-
 .../DurableClientReconnectDUnitTest.java        |   10 +-
 .../sockets/DurableClientStatsDUnitTest.java    |   10 +-
 .../sockets/DurableRegistrationDUnitTest.java   |    8 +-
 .../sockets/DurableResponseMatrixDUnitTest.java |   10 +-
 .../sockets/EventIDVerificationDUnitTest.java   |   10 +-
 .../ForceInvalidateEvictionDUnitTest.java       |   10 +-
 .../cache/tier/sockets/HAInterestBaseTest.java  |   64 +-
 .../tier/sockets/HAInterestPart2DUnitTest.java  |   12 +-
 .../sockets/HAStartupAndFailoverDUnitTest.java  |   81 +-
 .../tier/sockets/InterestListDUnitTest.java     |    4 +-
 .../sockets/InterestListEndpointDUnitTest.java  |   18 +-
 .../sockets/InterestListFailoverDUnitTest.java  |    6 +-
 .../sockets/InterestListRecoveryDUnitTest.java  |   10 +-
 .../sockets/InterestResultPolicyDUnitTest.java  |    4 +-
 .../tier/sockets/RedundancyLevelJUnitTest.java  |   14 +-
 .../sockets/RedundancyLevelPart1DUnitTest.java  |    4 +-
 .../sockets/RedundancyLevelPart3DUnitTest.java  |    6 +-
 .../tier/sockets/RedundancyLevelTestBase.java   |   51 +-
 .../tier/sockets/RegionCloseDUnitTest.java      |   14 +-
 ...erInterestBeforeRegionCreationDUnitTest.java |    6 +-
 .../sockets/RegisterInterestKeysDUnitTest.java  |    4 +-
 .../sockets/ReliableMessagingDUnitTest.java     |   24 +-
 .../internal/cache/tier/sockets/TestPut.java    |   53 -
 .../sockets/UpdatePropagationDUnitTest.java     |   10 +-
 ...UpdatesFromNonInterestEndPointDUnitTest.java |    4 +-
 ...ersalMembershipListenerAdapterDUnitTest.java |   32 +-
 .../pdx/PdxFormatterPutGetJUnitTest.java        |    4 +-
 .../test/java/dunit/DistributedTestCase.java    |    4 +-
 .../gemfire/codeAnalysis/excludedClasses.txt    |    2 +-
 .../sanctionedDataSerializables.txt             |   10 +-
 .../codeAnalysis/sanctionedSerializables.txt    |    9 -
 .../tier/sockets/RedundancyLevelJUnitTest.xml   |   21 +-
 252 files changed, 8011 insertions(+), 13441 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/admin/GemFireMemberStatus.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/GemFireMemberStatus.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/GemFireMemberStatus.java
index b1b91dd..e63cff5 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/GemFireMemberStatus.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/GemFireMemberStatus.java
@@ -37,7 +37,7 @@ import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionStatus;
 import com.gemstone.gemfire.internal.cache.RegionStatus;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 
 /**
@@ -539,7 +539,7 @@ public class GemFireMemberStatus implements Serializable {
       // The following method returns a map of client member id to a cache
       // client info. For now, keep track of the member ids in the set of
       // _connectedClients.
-      Map allConnectedClients = InternalBridgeMembership.getStatusForAllClientsIgnoreSubscriptionStatus();
+      Map allConnectedClients = InternalClientMembership.getStatusForAllClientsIgnoreSubscriptionStatus();
       Iterator allConnectedClientsIterator = allConnectedClients.values().iterator();
       while (allConnectedClientsIterator.hasNext()) {
         CacheClientStatus ccs = (CacheClientStatus) allConnectedClientsIterator.next();
@@ -549,7 +549,7 @@ public class GemFireMemberStatus implements Serializable {
       }
 
       // Get client queue sizes
-      Map clientQueueSize = getClientIDMap(InternalBridgeMembership.getClientQueueSizes());
+      Map clientQueueSize = getClientIDMap(InternalClientMembership.getClientQueueSizes());
       setClientQueueSizes(clientQueueSize);
       
       // Set server acceptor port (set it based on the first CacheServer)
@@ -590,11 +590,6 @@ public class GemFireMemberStatus implements Serializable {
 	  }
 
   protected void initializeClient() {
-    // There are several ways to detect a client:
-    // - is a loner
-    // - has regions that use BridgeWriters or BridgeLoaders
-    // This method uses the presence of a connection proxy or
-    // a pool on the PoolManager.
     Map poolMap = PoolManager.getAll();
     if (poolMap.size() == 0) {
       setIsClient(false);
@@ -608,7 +603,7 @@ public class GemFireMemberStatus implements Serializable {
       // the logical connections for that server will be 0. For now, keep track
       // of the keys (server names) of this map in the sets of _connectedServers
       // and _unconnectedServers.
-      Map connectedServers = InternalBridgeMembership.getConnectedServers();
+      Map connectedServers = InternalClientMembership.getConnectedServers();
       if (!connectedServers.isEmpty()) {
         Iterator connected = connectedServers.entrySet().iterator();
         while (connected.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberBridgeServer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberBridgeServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberBridgeServer.java
index 4cef371..b677cca 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberBridgeServer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberBridgeServer.java
@@ -10,11 +10,10 @@ package com.gemstone.gemfire.admin;
 import com.gemstone.gemfire.cache.server.ServerLoadProbe;
 
 /**
- * Administrative interface that represents a {@link
- * com.gemstone.gemfire.cache.util.BridgeServer BridgeServer} that
+ * Administrative interface that represents a CacheServer that
  * serves the contents of a system member's cache. 
  *
- * @see SystemMemberCache#addBridgeServer
+ * @see SystemMemberCache#addCacheServer
  *
  * @author David Whitlock
  * @since 4.0

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberCache.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberCache.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberCache.java
index 00749d6..14cf189 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberCache.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/SystemMemberCache.java
@@ -160,33 +160,6 @@ public interface SystemMemberCache {
   public void refresh();
 
   /**
-   * Adds a new, unstarted bridge server that will serve the contents
-   * of this cache.
-   *
-   * @see com.gemstone.gemfire.cache.Cache#addBridgeServer
-   *
-   * @since 4.0
-   * @deprecated as of 5.7 use {@link #addCacheServer} instead.
-   */
-  @Deprecated
-  public SystemMemberBridgeServer addBridgeServer()
-    throws AdminException;
-
-  /**
-   * Returns the bridge servers that run in this member's VM.  Note
-   * that this list will not be updated until {@link #refresh} is
-   * called.
-   *
-   * @see com.gemstone.gemfire.cache.Cache#getBridgeServers
-   *
-   * @since 4.0
-   * @deprecated as of 5.7 use {@link #getCacheServers} instead.
-   */
-  @Deprecated
-  public SystemMemberBridgeServer[] getBridgeServers()
-    throws AdminException;
-
-  /**
    * Adds a new, unstarted cache server that will serve the contents
    * of this cache to clients.
    *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthMonitor.java
index ca7f067..6058487 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/DistributedSystemHealthMonitor.java
@@ -373,7 +373,7 @@ class DistributedSystemHealthMonitor implements Runnable, GemFireVM {
     throw new UnsupportedOperationException(LocalizedStrings.DistributedSystemHealthMonitor_NOT_A_REAL_GEMFIREVM.toLocalizedString());
   }
 
-  public AdminBridgeServer addBridgeServer(CacheInfo cache)
+  public AdminBridgeServer addCacheServer(CacheInfo cache)
     throws AdminException {
 
     throw new UnsupportedOperationException(LocalizedStrings.DistributedSystemHealthMonitor_NOT_A_REAL_GEMFIREVM.toLocalizedString());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/SystemMemberCacheImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/SystemMemberCacheImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/SystemMemberCacheImpl.java
index d823c5e..88792d8 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/SystemMemberCacheImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/internal/SystemMemberCacheImpl.java
@@ -235,14 +235,9 @@ public class SystemMemberCacheImpl implements SystemMemberCache {
 
   public SystemMemberCacheServer addCacheServer()
     throws AdminException {
-    return (SystemMemberCacheServer)addBridgeServer();
-  }
-
-  public SystemMemberBridgeServer addBridgeServer()
-    throws AdminException {
 
-    AdminBridgeServer bridge = this.vm.addBridgeServer(this.info);
-    SystemMemberBridgeServer admin =
+    AdminBridgeServer bridge = this.vm.addCacheServer(this.info);
+    SystemMemberCacheServer admin =
       createSystemMemberBridgeServer(bridge);
     bridgeServers.put(bridge.getId(), admin);
     return admin;
@@ -280,14 +275,6 @@ public class SystemMemberCacheImpl implements SystemMemberCache {
     return (SystemMemberCacheServer[]) bridges.toArray(array);
   };
 
-  public SystemMemberBridgeServer[] getBridgeServers()
-    throws AdminException {
-    Collection bridges = getCacheServersCollection();
-    SystemMemberBridgeServer[] array =
-      new SystemMemberBridgeServer[bridges.size()];
-    return (SystemMemberBridgeServer[]) bridges.toArray(array);
-  }
-
   /**
    * Creates a new instance of <Code>SystemMemberBridgeServer</code>
    * with the given configuration.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/SystemMemberCacheJmxImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/SystemMemberCacheJmxImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/SystemMemberCacheJmxImpl.java
index 8a7c582..8376719 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/SystemMemberCacheJmxImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/admin/jmx/internal/SystemMemberCacheJmxImpl.java
@@ -285,19 +285,6 @@ implements com.gemstone.gemfire.admin.jmx.internal.ManagedResource {
   }
 
   /**
-   * Creates a new bridge server MBean and returns its
-   * <code>ObjectName</code>.
-   *
-   * @since 4.0
-   * @deprecated as of 5.7
-   */
-  @Deprecated
-  public ObjectName manageBridgeServer()
-    throws AdminException, MalformedObjectNameException {
-    return manageCacheServer();
-  }
-  
-  /**
    * Returns the MBean <code>ObjectName</code>s for all cache servers
    * that serve this cache to clients.
    *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/AttributesFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/AttributesFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/AttributesFactory.java
index fb3343f..7acd72a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/AttributesFactory.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/AttributesFactory.java
@@ -501,11 +501,6 @@ public class AttributesFactory<K,V> {
    */
   public void setCacheLoader(CacheLoader<K,V> cacheLoader)
   {
-    if (cacheLoader != null) {
-      if (AbstractRegion.isBridgeLoader(cacheLoader) && this.regionAttributes.getPoolName() != null) {
-        throw new IllegalStateException("A region with a pool name can not have a BridgeLoader or BridgeClient. Please use pools OR BridgeClient.");
-      }
-    }
     this.regionAttributes.cacheLoader = cacheLoader;
     this.regionAttributes.setHasCacheLoader(true);
   }
@@ -519,11 +514,6 @@ public class AttributesFactory<K,V> {
    */
   public void setCacheWriter(CacheWriter<K,V> cacheWriter)
   {
-    if (cacheWriter != null) {
-      if (AbstractRegion.isBridgeWriter(cacheWriter) &&  this.regionAttributes.getPoolName() != null) {
-        throw new IllegalStateException("A region with a pool name can not have a BridgeWriter or BridgeClient. Please use pools OR BridgeClient.");
-      }
-    }
     this.regionAttributes.cacheWriter = cacheWriter;
     this.regionAttributes.setHasCacheWriter(true);
   }
@@ -1285,8 +1275,6 @@ public class AttributesFactory<K,V> {
    * @param name the name of the connection pool to use; if <code>null</code>
    * or <code>""</code> then the connection pool is disabled for regions
    * using these attributes.
-   * @throws IllegalStateException if a cache loader or cache writer has already
-   * been set.
    * @since 5.7
    */
   public void setPoolName(String name) {
@@ -1294,17 +1282,6 @@ public class AttributesFactory<K,V> {
     if ("".equals(nm)) {
       nm = null;
     }
-    if (nm != null) {
-      // make sure a cache listener or writer has not already been installed
-      if (this.regionAttributes.getCacheLoader() != null 
-          && AbstractRegion.isBridgeLoader(this.regionAttributes.getCacheLoader())) {
-        throw new IllegalStateException("A region with a bridge loader can not have a pool name.");
-      }
-      if (this.regionAttributes.getCacheWriter() != null 
-          && AbstractRegion.isBridgeWriter(this.regionAttributes.getCacheWriter())) {
-        throw new IllegalStateException("A region with a bridge writer can not have a pool name.");
-      }
-    }
     this.regionAttributes.poolName = nm;
     this.regionAttributes.setHasPoolName(true);
     
@@ -1547,14 +1524,6 @@ public class AttributesFactory<K,V> {
         throw new IllegalStateException(LocalizedStrings.AttributesFactory_IF_THE_MEMBERSHIP_ATTRIBUTES_HAS_REQUIRED_ROLES_THEN_SCOPE_MUST_NOT_BE_LOCAL.toLocalizedString());
       }
     }
-    if (attrs.getPoolName() != null) {
-      if (attrs.getCacheLoader() != null && AbstractRegion.isBridgeLoader(attrs.getCacheLoader())) {
-        throw new IllegalStateException("A region with a pool name can not have a BridgeLoader or BridgeClient. Please use pools OR BridgeClient.");
-      }
-      if (attrs.getCacheWriter() != null && AbstractRegion.isBridgeWriter(attrs.getCacheWriter())) {
-        throw new IllegalStateException("A region with a pool name can not have a BridgeWriter or BridgeClient. Please use pools OR BridgeClient.");
-      }
-    }
     
     final PartitionAttributes pa = attrs.getPartitionAttributes();
     // Validations for PartitionRegion Attributes

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/Cache.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/Cache.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/Cache.java
index 72a5555..c6495d0 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/Cache.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/Cache.java
@@ -19,7 +19,6 @@ import com.gemstone.gemfire.cache.client.Pool;
 import com.gemstone.gemfire.cache.hdfs.HDFSStoreFactory;
 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;
@@ -237,18 +236,6 @@ public interface Cache extends GemFireCache {
   public void setSearchTimeout(int seconds);
 
   /**
-   * Creates a new bridge server with the default configuration.
-   *
-   * @see com.gemstone.gemfire.cache.util.BridgeLoader
-   * @see com.gemstone.gemfire.cache.util.BridgeWriter
-   *
-   * @since 4.0
-   * @deprecated as of 5.7 use {@link #addCacheServer} instead.
-   */
-  @Deprecated
-  public BridgeServer addBridgeServer();
-
-  /**
    * Creates a new cache server, with the default configuration,
    * that will allow clients to access this cache.
    * <p>For the default configuration see the constants in
@@ -260,20 +247,6 @@ public interface Cache extends GemFireCache {
   public CacheServer addCacheServer();
 
   /**
-   * Returns a collection of all of the <code>BridgeServer</code>s
-   * that can serve the contents of this <code>Cache</code>.
-   * <p>Since <code>5.7</code> this method returns a <code>List</code
-   * instead of a <code>Collection</code>.
-   *
-   * @see #addBridgeServer
-   *
-   * @since 4.0
-   * @deprecated as of 5.7 use {@link #getCacheServers} instead.
-   */
-  @Deprecated
-  public List<CacheServer> getBridgeServers();
-
-  /**
    * Returns a collection of all of the <code>CacheServer</code>s
    * that can serve the contents of this <code>Cache</code> to clients.
    *

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java
index 4922df9..8239ea2 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java
@@ -22,7 +22,6 @@ import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.client.Pool;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
-import com.gemstone.gemfire.cache.util.BridgeWriter;
 import com.gemstone.gemfire.cache.wan.GatewaySender;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
@@ -100,7 +99,7 @@ import com.gemstone.gemfire.security.GemFireSecurityException;
  * <p>
  * Notes:
  * <ul>
- * <li>DynamicRegionFactories in non-client VMs must not be configured with a BridgeWriter.
+ * <li>DynamicRegionFactories in non-client VMs must not be configured with a pool.
  * <li>If {@link #open()} is called before cache creation and the cache.xml has a dynamic-region-factory
  * element then the cache.xml will override the open call's configuration.
  * 
@@ -110,7 +109,7 @@ import com.gemstone.gemfire.security.GemFireSecurityException;
  * are shared by the parent and all its dynamic children
  * so make sure the callback is thread-safe and that its
  * {@link CacheCallback#close} implementation does not stop it from functioning.
- * However the products BridgeLoader, BridgeWriter, and all LRUAlgorithm instances will
+ * However the products LRUAlgorithm instances will
  * be cloned so that each dynamic Region has its own callback.
  * 
  * <li>The root Region name "DynamicRegions" is reserved. The factory creates a root Region of
@@ -223,7 +222,7 @@ public abstract class DynamicRegionFactory  {
     try {
       this.c = theCache;
       this.dynamicRegionList = theCache.getRegion(dynamicRegionListName);
-      final boolean isClient = this.config.getBridgeWriter() != null || this.config.getPoolName()!=null;
+      final boolean isClient = this.config.getPoolName()!=null;
       if (this.dynamicRegionList == null) {
         InternalRegionArguments ira = new InternalRegionArguments()
         .setDestroyLockFlag(true)
@@ -239,7 +238,6 @@ public abstract class DynamicRegionFactory  {
         }
 
         if (isClient) {
-          // BRIDGE CLIENT
           af.setScope(Scope.LOCAL);
           af.setDataPolicy(DataPolicy.NORMAL); //MirrorType(MirrorType.NONE);
           af.setStatisticsEnabled(true);
@@ -254,12 +252,6 @@ public abstract class DynamicRegionFactory  {
               }
               af.setPoolName(cpName);
             }
-          } else {
-            BridgeWriter bw = this.config.getBridgeWriter();
-            if (!bw.hasEstablishCallbackConnection()) {
-              throw new IllegalStateException(LocalizedStrings.DynamicRegionFactory_THE_CLIENT_POOL_OF_A_DYNAMICREGIONFACTORY_MUST_BE_CONFIGURED_WITH_ESTABLISHCALLBACKCONNECTION_SET_TO_TRUE.toLocalizedString());
-            }
-            af.setCacheWriter(bw);
           }
           ira.setInternalMetaRegion(new LocalMetaRegion(af.create(), ira));
         } else {
@@ -667,7 +659,7 @@ public abstract class DynamicRegionFactory  {
    * The default attributes are:
    * <ul>
    * <li>diskDir: <code>null</code>
-   * <li>bridgeWriter: <code>null</code>
+   * <li>poolName: <code>null</code>
    * <li>persistBackup: <code>true</code>
    * <li>registerInterest: <code>true</code>
    * </ul>
@@ -683,16 +675,10 @@ public abstract class DynamicRegionFactory  {
     public final File diskDir;
     /** Causes regions created by the factory to register interest in all keys in a corresponding server cache region */
     public final boolean registerInterest;
-    /** The {@link BridgeWriter} to be used by the factory to communicate with
-     * the factory in its server.
-     * Client factories must configure a BridgeWriter for their factory
-     * and it must be configured to establish a callback connection.
-     */
-    public final BridgeWriter bridgeWriter;
     
     /**
-     * The ${link Pool} to be used by the factory to communicate with 
-     * the server-side factory. Client factories may use this instead of a BridgeWriter 
+     * The ${link Pool} to be used by a client factory to communicate with 
+     * the server-side factory.
      */
     public final String poolName;
 
@@ -700,47 +686,15 @@ public abstract class DynamicRegionFactory  {
      * Creates a configuration with the default attributes.
      */
     public Config() {
-      this(null, null, !DISABLE_PERSIST_BACKUP);
-    }
-    /**
-     * Creates a configuration with the given attributes and defaults for other attributes.
-     * @deprecated use a pool name instead of a bridge writer
-     */
-    @Deprecated
-    public Config(File diskDir, BridgeWriter bridgeWriter) {
-      this(diskDir, bridgeWriter, !DISABLE_PERSIST_BACKUP);
+      this(null, null, !DISABLE_PERSIST_BACKUP, !DISABLE_REGISTER_INTEREST);
     }
-    
+
     /**
-     * Creates a configuration with the given attributes and defaults for other attributes.
-     * @deprecated use a pool name instead of a bridge writer
+     * Creates a configuration with defaults and the given diskDir and poolName.
      */
-    @Deprecated
-    public Config(File diskDir, BridgeWriter bridgeWriter, boolean persistBackup) {
-      this(diskDir, bridgeWriter, persistBackup, !DISABLE_REGISTER_INTEREST);
+    public Config(File diskDir, String poolName) {
+      this(diskDir, poolName, !DISABLE_PERSIST_BACKUP, !DISABLE_REGISTER_INTEREST);
     }
-    
-    
-    
-    /**
-     * Creates a configuration with the given attributes
-     * @deprecated use a pool name instead of a bridge writer
-     */
-    @Deprecated
-    public Config(
-      File diskDir,
-      BridgeWriter bridgeWriter,
-      boolean persistBackup,
-      boolean registerInterest)
-    {
-      this.registerInterest = registerInterest;
-      this.persistBackup = persistBackup;
-      this.diskDir = diskDir;
-      this.bridgeWriter = bridgeWriter;
-      this.poolName = null;
-    }
-    
-
     /**
      * Creates a configuration with the given attributes
      */
@@ -754,7 +708,43 @@ public abstract class DynamicRegionFactory  {
       this.persistBackup = persistBackup;
       this.diskDir = diskDir;
       this.poolName = poolName;
-      this.bridgeWriter = null;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((diskDir == null) ? 0 : diskDir.hashCode());
+      result = prime * result + (persistBackup ? 1231 : 1237);
+      result = prime * result + ((poolName == null) ? 0 : poolName.hashCode());
+      result = prime * result + (registerInterest ? 1231 : 1237);
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      Config other = (Config) obj;
+      if (diskDir == null) {
+        if (other.diskDir != null)
+          return false;
+      } else if (!diskDir.equals(other.diskDir))
+        return false;
+      if (persistBackup != other.persistBackup)
+        return false;
+      if (poolName == null) {
+        if (other.poolName != null)
+          return false;
+      } else if (!poolName.equals(other.poolName))
+        return false;
+      if (registerInterest != other.registerInterest)
+        return false;
+      return true;
     }
 
     /**
@@ -782,14 +772,6 @@ public abstract class DynamicRegionFactory  {
       return this.diskDir;
     }
     
-    /**
-     * Returns the {@link BridgeWriter} associated with the dynamic region factory.
-     * Returns null if there is no cache writer for dynamic regions.
-     * A cache writer will only exist if this is a client and the cache writer connects to a server.
-     */
-    public BridgeWriter getBridgeWriter() {
-      return this.bridgeWriter;
-    }
     
     /**
      * Returns the name of the {@link Pool} associated with the dynamic region factory.
@@ -803,7 +785,6 @@ public abstract class DynamicRegionFactory  {
     Config(Config conf) {
       this.diskDir = conf.diskDir;
       this.persistBackup = conf.persistBackup;
-      this.bridgeWriter = conf.bridgeWriter;
       this.registerInterest = conf.registerInterest;
       this.poolName = conf.poolName;
     }
@@ -814,7 +795,7 @@ public abstract class DynamicRegionFactory  {
       return;
     
     // Ignore the callback if it originated in this process (because the region
-    // will already have been created) and the event is not a bridge event
+    // will already have been created) and the event is not a client event
     if ( !event.isOriginRemote() && !event.isBridgeEvent() ) return;
     //
     DynamicRegionAttributes dra = (DynamicRegionAttributes)event.getNewValue();
@@ -913,7 +894,7 @@ public abstract class DynamicRegionFactory  {
     @Override
     protected boolean shouldNotifyBridgeClients()
     {
-      return getCache().getBridgeServers().size() > 0;
+      return getCache().getCacheServers().size() > 0;
     }
 
     // Over-ride the super behavior to perform the destruction of the dynamic region
@@ -1024,7 +1005,7 @@ public abstract class DynamicRegionFactory  {
     @Override
     final public boolean shouldNotifyBridgeClients()
     {
-      return getCache().getBridgeServers().size() > 0;
+      return getCache().getCacheServers().size() > 0;
     }    
    
     // Over-ride the super behavior to perform the destruction of the dynamic region

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/PoolFactory.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/PoolFactory.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/PoolFactory.java
index 547971f..5c29938 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/PoolFactory.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/PoolFactory.java
@@ -17,7 +17,7 @@ import com.gemstone.gemfire.cache.query.*; // for javadocs
  * {@link Pool}.
  * <p>Every pool needs to have at least one {@link #addLocator locator} or {@link #addServer server} added
  * to it. Locators should be added unless direct connections to
- * bridge servers are desired.
+ * cache servers are desired.
  * <p>The setter methods are used to specify
  * non-default values for the other pool properties.
  * <p>Once it is configured {@link #create}
@@ -290,7 +290,7 @@ public interface PoolFactory {
    * 
    * These pings are used by the server to monitor the health of
    * the client. Make sure that the pingInterval is less than the 
-   * maximum time between pings allowed by the bridge server.
+   * maximum time between pings allowed by the cache server.
    * @param pingInterval The amount of time in milliseconds between
    * pings.
    * @return a reference to <code>this</code>
@@ -325,7 +325,7 @@ public interface PoolFactory {
   /**
    * Add a locator, given its host and port, to this factory.
    * The locator must be a server locator and will be used to discover other running
-   * bridge servers and locators.
+   * cache servers and locators.
    * Note that if the host is unknown at the time of this call
    * the locator will still be added. When the pool is used for
    * an operation if the host is still unknown an exception will
@@ -341,7 +341,7 @@ public interface PoolFactory {
 
   /**
    * Add a server, given its host and port, to this factory.
-   * The server must be a bridge server and this client will
+   * The server must be a cache server and this client will
    * directly connect to without consulting a server locator.
    * Note that if the host is unknown at the time of this call
    * the server will still be added. When the pool is used for
@@ -392,7 +392,7 @@ public interface PoolFactory {
   
   /**
    * Sets the interval in milliseconds
-   * to wait before sending acknowledgements to the bridge server for
+   * to wait before sending acknowledgements to the cache server for
    * events received from the server subscriptions.
    * 
    * @param ackInterval number of milliseconds to wait before sending event

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AbstractOp.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AbstractOp.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AbstractOp.java
index ab11517..32f184e 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AbstractOp.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/AbstractOp.java
@@ -19,7 +19,7 @@ import com.gemstone.gemfire.cache.client.ServerOperationException;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.PutAllPartialResultException;
 import com.gemstone.gemfire.internal.cache.TXManagerImpl;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/BridgePoolImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/BridgePoolImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/BridgePoolImpl.java
deleted file mode 100644
index 25abf2d..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/BridgePoolImpl.java
+++ /dev/null
@@ -1,479 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-package com.gemstone.gemfire.cache.client.internal;
-
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.CancelException;
-import com.gemstone.gemfire.cache.CacheLoaderException;
-import com.gemstone.gemfire.cache.CacheWriterException;
-import com.gemstone.gemfire.cache.NoSubscriptionServersAvailableException;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.client.Pool;
-import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.client.ServerConnectivityException;
-import com.gemstone.gemfire.cache.client.SubscriptionNotEnabledException;
-import com.gemstone.gemfire.cache.client.internal.GetOp.GetOpImpl;
-import com.gemstone.gemfire.cache.util.BridgeWriterException;
-import com.gemstone.gemfire.cache.util.EndpointDoesNotExistException;
-import com.gemstone.gemfire.cache.util.EndpointExistsException;
-import com.gemstone.gemfire.cache.util.EndpointInUseException;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.internal.cache.PoolFactoryImpl;
-import com.gemstone.gemfire.internal.cache.PoolManagerImpl;
-import com.gemstone.gemfire.internal.cache.tier.ConnectionProxy;
-import com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl;
-import com.gemstone.gemfire.internal.logging.LogService;
-
-/**
- * A pool for use by the old BridgeLoader/BridgeWriter.
- * This class can go away once we drop these deprecated classes
- * 
- * @author darrel
- * @since 5.7
- */
-@SuppressWarnings("deprecation")
-public class BridgePoolImpl extends PoolImpl implements ConnectionProxy {
-  private static final Logger logger = LogService.getLogger();
-  
-  private static final AtomicInteger ID_COUNTER = new AtomicInteger();
-  public static final int DEFAULT_CONNECTIONSPERSERVER = 1;
-  public static final int DEFAULT_HANDSHAKE_TIMEOUT = AcceptorImpl.DEFAULT_HANDSHAKE_TIMEOUT_MS;
-  public static final String DEFAULT_LBPOLICY = LBPolicy.STICKY_PROPERTY_NAME;
-  
-  //this field is only set to true when the cache is closing with
-  //keep alive set to true.
-  private boolean keepAlive;
-
-  private static int getBridgePoolId() {
-    return ID_COUNTER.incrementAndGet();
-  }
-  
-  public static BridgePoolImpl create(Properties props, boolean usedByBridgeWriter) {
-    return create(props, usedByBridgeWriter, false/*usedByGateway*/);
-  }
-
-  public static BridgePoolImpl create(Properties props, boolean usedByBridgeWriter,
-                                      boolean usedByGateway) {
-    
-    String name = (usedByGateway ? "GatewayPool-" : "BridgePool-") + getBridgePoolId();
-    PoolFactoryImpl pf = (PoolFactoryImpl)PoolManager.createFactory();
-    pf.init(props, usedByBridgeWriter, usedByGateway);
-    BridgePoolImpl result = (BridgePoolImpl)pf.create(name);
-    if (!result.isDestroyed()) {
-      result.attach();
-    }
-    return result;
-  }
-  /**
-   * Should only be called by PoolFactoryImpl#create.
-   * All other creators should use the static create method
-   */
-  public BridgePoolImpl(PoolManagerImpl pm, String name, Pool attributes) {
-    super(pm, name, attributes);
-    finishCreate(pm); // do this last since we are escaping the constructor
-  }
-  public static void loadEmergencyClasses() {
-    // nyi
-  }
-
-  ////////////////// ConnectionProxy methods ////////////////////
-  /**
-   * Initializes this <code>ConnectionProxy</code> according to the
-   * given properties.
-   */
-  public void initialize(Properties p) {
-    throw new IllegalStateException("nyi");
-  }
-
-
-
-  public void finalizeProxy() {
-    detach();
-    if (getAttachCount() <= 0) {
-      destroy(keepAlive);
-    }
-  }
-
-
-  /**
-   * Returns the load balancing policy in effect for this connection
-   * proxy.
-   */
-  public String getLBPolicy() {
-    if (getThreadLocalConnections()) {
-      return "Sticky";
-    } else {
-      return "RoundRobin";
-    }
-  }
-
-
-  /**
-   * Returns the number of milliseconds to wait before re-connecting
-   * to a dead server.
-   */
-  public int getRetryInterval() {
-    return (int)getPingInterval();
-  }
-
-
-  /**
-   * Closes this connection proxy and all of its connections
-   */
-  public void close() {
-    if (logger.isDebugEnabled()) {
-      logger.debug("BridgePoolImpl - closing");
-    }
-    finalizeProxy();
-  }
-
-
-  /**
-   * Returned true if this ConnectionProxy has been initialized and not closed.
-   */
-  public boolean isOpen() {
-    return !isDestroyed();
-  }
-
-
-  /**
-   * Update bookkeeping on this proxy associated with the loss of a region.
-   * In particular, remove all region interests.
-   */
-  public void detachRegion(Region r) {
-    // nyi
-  }
-
-
-  /**
-   * Returns the number of {@link Connection}s that should be created
-   * to every cache server.
-   */
-  public int getConnectionsPerServer() {
-    return getMinConnections();
-  }
-
-
-  /**
-   * Notes that the server with the given name is unavailable
-   */
-  public void setServerUnavailable(String name) {
-    throw new IllegalStateException("nyi");
-  }
-
-
-  /**
-   * Notes that the server with the given name is available
-   */
-  public void setServerAvailable(String name) {
-    throw new IllegalStateException("nyi");
-  }
-
-
-  /**
-   * Stops this connection proxy and
-   */
-  public void terminate() {
-    finalizeProxy();
-  }
-
-
-  /**
-   * Releases the connection associated with the current thread
-   */
-  public void release() {
-    // nyi
-  }
-
-  /**
-   * Returns value of establishCallbackConnection property.
-   * @since 4.2.3
-   */
-  public boolean getEstablishCallbackConnection() {
-    return getSubscriptionEnabled();
-  }
-
-
-  /**
-   * Add an <code>Endpoint</code> to the known <code>Endpoint</code>s.
-   *
-   * @param name The name of the endpoint to add
-   * @param host The host name or ip address of the endpoint to add
-   * @param port The port of the endpoint to add
-   *
-   * @throws EndpointExistsException if the <code>Endpoint</code> to be
-   * added already exists.
-   *
-   * @since 5.0.2
-   */
-  public void addEndpoint(String name, String host, int port)
-  throws EndpointExistsException {
-    ((ExplicitConnectionSourceImpl)getConnectionSource()).addEndpoint(host,port);
-  }
-
-
-  /**
-   * Remove an <code>Endpoint</code> from the dead <code>Endpoint</code>s.
-   * The specified <code>Endpoint</code> must be dead.
-   *
-   * @param name The name of the endpoint to remove
-   * @param host The host name or ip address of the endpoint to remove
-   * @param port The port of the endpoint to remove
-   *
-   * @throws EndpointDoesNotExistException if the <code>Endpoint</code> to be
-   * removed doesn't exist.
-   *
-   * @since 5.0.2
-   */
-  public void removeEndpoint(String name, String host, int port)
-  throws EndpointDoesNotExistException,EndpointInUseException {
-    ((ExplicitConnectionSourceImpl)getConnectionSource()).removeEndpoint(host,port);
-  }
-
-
-  /**
-   * @return Returns the redundancy number
-   * @since 5.1
-   */
-  public int getRedundancyLevel() {
-    return getSubscriptionRedundancy();
-  }
-
-  /**
-   * The configurable expiry time of last received sequence ID
-   *
-   * @return The configurable expiry time of last received sequence ID
-   */
-  public long getMessageTrackingTimeout() {
-    return getSubscriptionMessageTrackingTimeout();
-  }
-
-  public void reuse() {
-    attach();
-  }
-  
-  
-  public static boolean isLoaderOp(Op op) {
-    return op instanceof GetOpImpl;
-  }
-
-  private RuntimeException transformException(RuntimeException ex, Op op) {
-    if(isLoaderOp(op)) {
-      if (ex instanceof SubscriptionNotEnabledException) {
-        return new CacheLoaderException("establishCallbackConnection must be set to true", ex);
-      } else if (ex instanceof CacheLoaderException) {
-        return ex;
-      } else if (ex instanceof CancelException) {
-        return ex;
-      } else if (ex instanceof ServerConnectivityException && ex.getCause() != null) {
-        return new CacheLoaderException(ex.getCause());
-      } else {
-        return new CacheLoaderException(ex);
-      }
-    }
-    else {
-      if (ex instanceof SubscriptionNotEnabledException) {
-        return new BridgeWriterException("establishCallbackConnection must be set to true", ex);
-      } else if (ex instanceof CacheWriterException) {
-        return ex;
-      } else if (ex instanceof CancelException) {
-        return ex;
-      } else if (ex instanceof ServerConnectivityException && ex.getCause() != null) {
-        return new BridgeWriterException(ex.getCause());
-      } else {
-        return new BridgeWriterException(ex);
-      }
-    }
-  }
-  
-  @Override
-  public Object execute(Op op) {
-    try {
-      return super.execute(op);
-    } catch (RuntimeException ex) {
-      throw transformException(ex, op);
-    }
-  }
-
-  @Override
-  public Object executeOn(ServerLocation server, Op op) {
-    try {
-      return super.executeOn(server, op);
-    } catch (RuntimeException ex) {
-      throw transformException(ex, op);
-    }
-  }
-  
-  @Override
-  public Object executeOn(Connection con, Op op) {
-    try {
-      return super.executeOn(con, op);
-    } catch (RuntimeException ex) {
-      throw transformException(ex, op);
-    }
-  }
-
-  @Override
-  public Object executeOn(Connection con, Op op, boolean timeoutFatal) {
-    try {
-      return super.executeOn(con, op, timeoutFatal);
-    } catch (RuntimeException ex) {
-      throw transformException(ex, op);
-    }
-  }
-
-  @Override
-  public Object executeOnQueuesAndReturnPrimaryResult(Op op) {
-    try {
-      return super.executeOnQueuesAndReturnPrimaryResult(op);
-    } catch (RuntimeException ex) {
-      throw transformException(ex, op);
-    }
-  }
-
-  @Override
-  public void executeOnAllQueueServers(Op op)
-    throws NoSubscriptionServersAvailableException, SubscriptionNotEnabledException {
-    try {
-      super.executeOnAllQueueServers(op);
-    } catch (RuntimeException ex) {
-      throw transformException(ex, op);
-    }
-  }
-
-  @Override
-  public Object executeOnPrimary(Op op) {
-    try {
-      return super.executeOnPrimary(op);
-    } catch (RuntimeException ex) {
-      throw transformException(ex, op);
-    }
-  }
-  public void setKeepAlive(boolean keepAlive) {
-    this.keepAlive = keepAlive;
-  }
-  
-  /** ******** INNER CLASSES ************************* */
-  public static class LBPolicy
-  {
-    public static final String STICKY_PROPERTY_NAME = "Sticky";
-
-    public static final String RANDOMSTICKY_PROPERTY_NAME = "RandomSticky";
-
-    public static final String ROUNDROBIN_PROPERTY_NAME = "RoundRobin";
-
-    public static final String RANDOM_PROPERTY_NAME = "Random";
-
-    public static final String APPASSISTED_PROPERTY_NAME = "AppAssisted";
-
-    public static final int STICKY = 0;
-
-    public static final int ROUNDROBIN = 1;
-
-    public static final int APPASSISTED = 2;
-
-    public static final int RANDOM = 3;
-
-    public static final int RANDOMSTICKY = 4;
-
-    public final int thePolicy;
-
-    public LBPolicy(String name) {
-      if (name.equalsIgnoreCase(STICKY_PROPERTY_NAME)) {
-        this.thePolicy = STICKY;
-      }
-      else if (name.equalsIgnoreCase(ROUNDROBIN_PROPERTY_NAME)) {
-        this.thePolicy = ROUNDROBIN;
-      }
-      else if (name.equalsIgnoreCase(APPASSISTED_PROPERTY_NAME)) {
-        this.thePolicy = APPASSISTED;
-      }
-      else if (name.equalsIgnoreCase(RANDOM_PROPERTY_NAME)) {
-        this.thePolicy = RANDOM;
-      }
-      else if (name.equalsIgnoreCase(RANDOMSTICKY_PROPERTY_NAME)) {
-        this.thePolicy = RANDOMSTICKY;
-      }
-      else {
-        this.thePolicy = STICKY; // DEFAULT
-      }
-    }
-
-    public int getPolicy()
-    {
-      return this.thePolicy;
-    }
-
-    public boolean isSticky() {
-      return getPolicy() == STICKY || getPolicy() == RANDOMSTICKY;
-    }
-
-    public boolean isRandom() {
-      return getPolicy() == RANDOM || getPolicy() == RANDOMSTICKY;
-    }
-
-    public String getPolicyPropertyName(int pol)
-    {
-      String retStr;
-      switch (pol) {
-      case STICKY:
-        retStr = STICKY_PROPERTY_NAME;
-        break;
-      case ROUNDROBIN:
-        retStr = ROUNDROBIN_PROPERTY_NAME;
-        break;
-      case APPASSISTED:
-        retStr = APPASSISTED_PROPERTY_NAME;
-        break;
-      case RANDOM:
-        retStr = RANDOM_PROPERTY_NAME;
-        break;
-      case RANDOMSTICKY:
-        retStr = RANDOMSTICKY_PROPERTY_NAME;
-        break;
-      default:
-        return Integer.toString(pol);
-      }
-      return retStr;
-    }
-
-    @Override
-    public boolean equals(Object obj)
-    {
-      if (obj == this) {
-        return true;
-      }
-
-      if (obj instanceof LBPolicy) {
-        LBPolicy other = (LBPolicy)obj;
-        return this.thePolicy == other.thePolicy;
-      }
-      else {
-        return false;
-      }
-    }
-
-    @Override
-    public int hashCode()
-    {
-      return this.thePolicy;
-    }
-
-    @Override
-    public String toString()
-    {
-      return getPolicyPropertyName(this.thePolicy);
-    }
-  }
-
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/BridgeServerLoadMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/BridgeServerLoadMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/BridgeServerLoadMessage.java
deleted file mode 100644
index a419216..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/BridgeServerLoadMessage.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-package com.gemstone.gemfire.cache.client.internal;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.cache.server.ServerLoad;
-import com.gemstone.gemfire.distributed.Locator;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.InternalLocator;
-import com.gemstone.gemfire.distributed.internal.SerialDistributionMessage;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.distributed.internal.ServerLocator;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-
-/**
- * A message from bridge server to locator to update the locator
- * with new load information from the bridge server.
- * Also includes the id of any clients whose estimate is no
- * longer needed on the server-locator.
- * @author dsmith
- * @since 5.7
- *
- */
-public class BridgeServerLoadMessage extends SerialDistributionMessage {
-  protected ServerLoad load;
-  protected ServerLocation location;
-  protected ArrayList clientIds;
-
-  public BridgeServerLoadMessage() {
-    super();
-  }
-  
-  public BridgeServerLoadMessage(ServerLoad load, ServerLocation location,
-                                 ArrayList clientIds) {
-    super();
-    this.load = load;
-    this.location = location;
-    this.clientIds = clientIds;
-  }
-
-  @Override
-  protected void process(DistributionManager dm) {
-    updateLocalLocators();
-  }
-
-  public void updateLocalLocators() {
-    List locators = Locator.getLocators();
-    for (int i=0; i < locators.size(); i++) {
-      InternalLocator l = (InternalLocator)locators.get(i);
-      ServerLocator serverLocator = l.getServerLocatorAdvisee();
-      if(serverLocator != null) {
-        serverLocator.updateLoad(location, load, this.clientIds);
-      }
-    }
-  }
-  
-  
-
-  public int getDSFID() {
-   return BRIDGE_SERVER_LOAD_MESSAGE;
-  }
-
-  @Override
-  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    super.fromData(in);
-    load = new ServerLoad();
-    InternalDataSerializer.invokeFromData(load, in);
-    location = new ServerLocation();
-    InternalDataSerializer.invokeFromData(location, in);
-    this.clientIds = DataSerializer.readArrayList(in);
-  }
-
-  @Override
-  public void toData(DataOutput out) throws IOException {
-    super.toData(out);
-    InternalDataSerializer.invokeToData(load, out);
-    InternalDataSerializer.invokeToData(location, out);
-    DataSerializer.writeArrayList(this.clientIds, out);
-  }
-
-  @Override
-  protected Object clone() throws CloneNotSupportedException {
-    return super.clone();
-  }
-  
-  
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/CacheServerLoadMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/CacheServerLoadMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/CacheServerLoadMessage.java
new file mode 100644
index 0000000..e4f63dc
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/CacheServerLoadMessage.java
@@ -0,0 +1,99 @@
+/*=========================================================================
+ * Copyright (c) 2002-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
+ * more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.cache.client.internal;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.server.ServerLoad;
+import com.gemstone.gemfire.distributed.Locator;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.SerialDistributionMessage;
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+import com.gemstone.gemfire.distributed.internal.ServerLocator;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
+
+/**
+ * A message from a server to a locator to update the locator
+ * with new load information from the server.
+ * Also includes the id of any clients whose estimate is no
+ * longer needed on the server-locator.
+ * @author dsmith
+ * @since 5.7
+ *
+ */
+public class CacheServerLoadMessage extends SerialDistributionMessage {
+  protected ServerLoad load;
+  protected ServerLocation location;
+  protected ArrayList clientIds;
+
+  public CacheServerLoadMessage() {
+    super();
+  }
+  
+  public CacheServerLoadMessage(ServerLoad load, ServerLocation location,
+                                 ArrayList clientIds) {
+    super();
+    this.load = load;
+    this.location = location;
+    this.clientIds = clientIds;
+  }
+
+  @Override
+  protected void process(DistributionManager dm) {
+    updateLocalLocators();
+  }
+
+  public void updateLocalLocators() {
+    List locators = Locator.getLocators();
+    for (int i=0; i < locators.size(); i++) {
+      InternalLocator l = (InternalLocator)locators.get(i);
+      ServerLocator serverLocator = l.getServerLocatorAdvisee();
+      if(serverLocator != null) {
+        serverLocator.updateLoad(location, load, this.clientIds);
+      }
+    }
+  }
+  
+  
+
+  public int getDSFID() {
+   return CACHE_SERVER_LOAD_MESSAGE;
+  }
+
+  @Override
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    super.fromData(in);
+    load = new ServerLoad();
+    InternalDataSerializer.invokeFromData(load, in);
+    location = new ServerLocation();
+    InternalDataSerializer.invokeFromData(location, in);
+    this.clientIds = DataSerializer.readArrayList(in);
+  }
+
+  @Override
+  public void toData(DataOutput out) throws IOException {
+    super.toData(out);
+    InternalDataSerializer.invokeToData(load, out);
+    InternalDataSerializer.invokeToData(location, out);
+    DataSerializer.writeArrayList(this.clientIds, out);
+  }
+
+  @Override
+  protected Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+  
+  
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/DestroyOp.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/DestroyOp.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/DestroyOp.java
index 7dc0eaa..dbcc7ed 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/DestroyOp.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/DestroyOp.java
@@ -15,7 +15,6 @@ import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.cache.client.AllConnectionsInUseException;
 import com.gemstone.gemfire.cache.client.ServerConnectivityException;
 import com.gemstone.gemfire.cache.client.ServerOperationException;
-import com.gemstone.gemfire.cache.util.BridgeWriterException;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
@@ -80,10 +79,6 @@ public class DestroyOp {
           }
           cms.removeBucketServerLocation(server);
         }
-        catch (BridgeWriterException e) {
-          if (e.getCause() instanceof ServerConnectivityException)
-            cms.removeBucketServerLocation(server);
-        }
       }
     }
     return pool.execute(op);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java
index 5bf3a48..e4741de 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java
@@ -26,7 +26,7 @@ import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.DummyStatisticsFactory;
 import com.gemstone.gemfire.internal.cache.PoolStats;
 import com.gemstone.gemfire.internal.cache.execute.TransactionFunctionService;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.logging.LogService;
 
 /**
@@ -274,7 +274,7 @@ public class EndpointManagerImpl implements EndpointManager {
         return;
       }
       //logger.warn("EMANFIRE:CRASH:"+endpoint.getLocation());
-      InternalBridgeMembership.notifyCrashed(endpoint.getMemberId(), false);
+      InternalClientMembership.notifyCrashed(endpoint.getMemberId(), false);
     }
 
     public void endpointNoLongerInUse(Endpoint endpoint) {
@@ -282,7 +282,7 @@ public class EndpointManagerImpl implements EndpointManager {
         return;
       }
       //logger.warn("EMANFIRE:LEFT:"+endpoint.getLocation());
-      InternalBridgeMembership.notifyLeft(endpoint.getMemberId(), false);
+      InternalClientMembership.notifyLeft(endpoint.getMemberId(), false);
     }
 
     public void endpointNowInUse(Endpoint endpoint) {
@@ -290,7 +290,7 @@ public class EndpointManagerImpl implements EndpointManager {
         return;
       }
       //logger.warn("EMANFIRE:JOIN:"+endpoint.getLocation()+" mid:"+endpoint.getMemberId(),new Exception());
-      InternalBridgeMembership.notifyJoined(endpoint.getMemberId(), false);
+      InternalClientMembership.notifyJoined(endpoint.getMemberId(), false);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ExplicitConnectionSourceImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ExplicitConnectionSourceImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ExplicitConnectionSourceImpl.java
index a1bdc16..9e2b6ca 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ExplicitConnectionSourceImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/ExplicitConnectionSourceImpl.java
@@ -7,9 +7,7 @@
  */
 package com.gemstone.gemfire.cache.client.internal;
 
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
-import java.net.UnknownHostException; 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
@@ -19,9 +17,6 @@ import java.util.Set;
 
 import org.apache.logging.log4j.Logger;
 
-import com.gemstone.gemfire.cache.util.EndpointDoesNotExistException;
-import com.gemstone.gemfire.cache.util.EndpointExistsException;
-import com.gemstone.gemfire.cache.util.EndpointInUseException;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerQueueStatus;
@@ -64,15 +59,18 @@ public class ExplicitConnectionSourceImpl implements ConnectionSource {
     this.serverList = Collections.unmodifiableList(serverList);
   }
 
+  @Override
   public synchronized void start(InternalPool pool) {
     this.pool = pool;
     pool.getStats().setInitialContacts(serverList.size());
   }
   
+  @Override
   public void stop() {
     //do nothing
   }
 
+  @Override
   public ServerLocation findReplacementServer(ServerLocation currentServer, Set/*<ServerLocation>*/ excludedServers) {
     // at this time we always try to find a server other than currentServer
     // and if we do return it. Otherwise return null;
@@ -85,6 +83,7 @@ public class ExplicitConnectionSourceImpl implements ConnectionSource {
     return findServer(excludedPlusCurrent);
   }
   
+  @Override
   public synchronized ServerLocation findServer(Set excludedServers) {
     if(PoolImpl.TEST_DURABLE_IS_NET_DOWN) {
       return null;
@@ -109,6 +108,7 @@ public class ExplicitConnectionSourceImpl implements ConnectionSource {
    * connect to every server in the system to find where our durable
    * queue lives.
    */
+  @Override
   public synchronized List findServersForQueue(Set excludedServers,
       int numServers, ClientProxyMembershipID proxyId, boolean findDurableQueue) {
     if(PoolImpl.TEST_DURABLE_IS_NET_DOWN) {
@@ -124,80 +124,7 @@ public class ExplicitConnectionSourceImpl implements ConnectionSource {
     }
   }
   
-  /**
-   * Remove an endpoint from this connection source.
-   * 
-   * @param host
-   * @param port
-   * @throws EndpointDoesNotExistException if the <code>Endpoint</code> to be
-   * removed doesn't exist.
-   */
-  public synchronized void removeEndpoint(String host,int port) throws EndpointInUseException,EndpointDoesNotExistException {
-    serverList = new ArrayList(serverList);
-    Iterator it = serverList.iterator();
-    boolean found = false;
-    host = lookupHostName(host);
-    while(it.hasNext()) {
-      ServerLocation loc = (ServerLocation)it.next();
-      if(loc.getHostName().equalsIgnoreCase(host)) {
-        if(loc.getPort()==port) {
-          EndpointManager em = pool.getEndpointManager();
-          if(em.getEndpointMap().containsKey(loc)) {
-            throw new EndpointInUseException("Endpoint in use cannot be removed:"+loc);
-          } else {
-            it.remove();
-            found = true;
-          }
-        }
-      }
-    }
-    serverList = Collections.unmodifiableList(serverList);
-    if(!found) {
-      throw new EndpointDoesNotExistException("endpointlist:"+serverList,host,port);
-    }
-  }
-  
-  /**
-   * Add an endpoint to this connection source.
-   * 
-   * @param host
-   * @param port
-   * @throws EndpointExistsException if the <code>Endpoint</code> to be
-   * added already exists.
-   */
-  public synchronized void addEndpoint(String host,int port) throws EndpointExistsException {
-    Iterator it = serverList.iterator();
-    host = lookupHostName(host);
-    while(it.hasNext()) {
-      ServerLocation loc = (ServerLocation)it.next();
-      if(loc.getHostName().equalsIgnoreCase(host)) {
-        if(loc.getPort()==port) {
-          throw new EndpointExistsException("Endpoint already exists host="+host+" port="+port);
-        }
-      }
-    }
-    serverList = new ArrayList(serverList);
-    serverList.add(new ServerLocation(host,port));
-    serverList = Collections.unmodifiableList(serverList);
-  }
- 
-  /**
-   * When we create an ExplicitConnectionSource, we convert a the hostname of an
-   * endpoint from a string to an InetAddress and back. This method duplicates
-   * that process for endpoints that are added or removed after the fact.
-   */
-  private String lookupHostName(String host) {
-    try {
-      InetAddress hostAddr = InetAddress.getByName(host);
-      host = hostAddr.getHostName();
-    } catch (UnknownHostException cause) {
-      IllegalArgumentException ex = new IllegalArgumentException("Unknown host " + host);
-      ex.initCause(cause);
-      throw ex;
-    }
-    return host;
-  } 
-
+  @Override
   public boolean isBalanced() {
     return false;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/GetOp.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/GetOp.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/GetOp.java
index 019da2e..a187bcb 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/GetOp.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/GetOp.java
@@ -15,7 +15,6 @@ import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.cache.client.AllConnectionsInUseException;
 import com.gemstone.gemfire.cache.client.ServerConnectivityException;
 import com.gemstone.gemfire.cache.client.ServerOperationException;
-import com.gemstone.gemfire.cache.util.BridgeWriterException;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
@@ -81,10 +80,6 @@ public class GetOp {
             }
             cms.removeBucketServerLocation(server);
           }
-          catch (BridgeWriterException e) {
-            if (e.getCause() instanceof ServerConnectivityException)
-              cms.removeBucketServerLocation(server);
-          }
           catch (CacheLoaderException e) {
             if (e.getCause() instanceof ServerConnectivityException)
               cms.removeBucketServerLocation(server);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PoolImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PoolImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PoolImpl.java
index 851c5c8..d211417 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PoolImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PoolImpl.java
@@ -1204,36 +1204,36 @@ public class PoolImpl implements InternalPool {
   
   ///////////////////// start test hooks ///////////////////////
   /**
-   * A debug flag used for testing used in BridgeObserver
+   * A debug flag used for testing used in ClientServerObserver
    */
   public static volatile boolean AFTER_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = false;
 
   /**
-   * A debug flag used for testing used in BridgeObserver
+   * A debug flag used for testing used in ClientServerObserver
    */
   public static volatile boolean BEFORE_REGISTER_CALLBACK_FLAG = false;
 
   /**
-   * A debug flag used for testing used in BridgeObserver
+   * A debug flag used for testing used in ClientServerObserver
    */
   public static volatile boolean BEFORE_RECOVER_INTEREST_CALLBACK_FLAG = false;
 
   /**
-   * A debug flag used for testing used in BridgeObserver
+   * A debug flag used for testing used in ClientServerObserver
    */
   public static volatile boolean AFTER_REGISTER_CALLBACK_FLAG = false;
 
   /**
-   * A debug flag used for testing used in BridgeObserver
+   * A debug flag used for testing used in ClientServerObserver
    */
   public static volatile boolean BEFORE_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = false;
 
   /**
-   * A debug flag used for testing used in BridgeObserver
+   * A debug flag used for testing used in ClientServerObserver
    */
   public static volatile boolean BEFORE_SENDING_CLIENT_ACK_CALLBACK_FLAG = false;
   /**
-   * A debug flag used for testing used in BridgeObserver
+   * A debug flag used for testing used in ClientServerObserver
    */  
   public static volatile boolean AFTER_QUEUE_DESTROY_MESSAGE_FLAG = false;
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java
index 35822ed..05caa9c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/PutOp.java
@@ -14,7 +14,6 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.AllConnectionsInUseException;
 import com.gemstone.gemfire.cache.client.ServerConnectivityException;
 import com.gemstone.gemfire.cache.client.ServerOperationException;
-import com.gemstone.gemfire.cache.util.BridgeWriterException;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
@@ -91,10 +90,6 @@ public class PutOp {
           }
           cms.removeBucketServerLocation(server);
         }
-        catch (BridgeWriterException e) {
-          if (e.getCause() instanceof ServerConnectivityException)
-            cms.removeBucketServerLocation(server);
-        }
       }
     }
     return pool.execute(op);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueManagerImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueManagerImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueManagerImpl.java
index f0bf9e9..1d6597f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueManagerImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueManagerImpl.java
@@ -48,8 +48,8 @@ import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.Assert;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.InterestType;
@@ -733,7 +733,7 @@ public class QueueManagerImpl implements QueueManager {
   private boolean promoteBackupCnxToPrimary(QueueConnectionImpl cnx) {
     boolean result = false;
     if (PoolImpl.BEFORE_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG) {
-      BridgeObserver bo = BridgeObserverHolder.getInstance();
+      ClientServerObserver bo = ClientServerObserverHolder.getInstance();
       bo.beforePrimaryIdentificationFromBackup();
     }
     try {
@@ -752,7 +752,7 @@ public class QueueManagerImpl implements QueueManager {
         MakePrimaryOp.execute(pool, cnx, haveSentClientReady);
         result = true;
         if (PoolImpl.AFTER_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG) {
-          BridgeObserver bo = BridgeObserverHolder.getInstance();
+          ClientServerObserver bo = ClientServerObserverHolder.getInstance();
           bo.afterPrimaryIdentificationFromBackup(cnx.getServer());
         }
       }
@@ -888,7 +888,7 @@ public class QueueManagerImpl implements QueueManager {
         logger.debug("SubscriptionManager redundancy satisfier - Switched backup server to primary: {}", newPrimary.getEndpoint());
       }
       if (PoolImpl.AFTER_PRIMARY_RECOVERED_CALLBACK_FLAG) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.afterPrimaryRecovered(newPrimary.getServer());
       }
      
@@ -924,7 +924,7 @@ public class QueueManagerImpl implements QueueManager {
       }
 
       if (newPrimary != null && PoolImpl.AFTER_PRIMARY_RECOVERED_CALLBACK_FLAG) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.afterPrimaryRecovered(newPrimary.getServer());
       }
       printPrimaryNotFoundError = true;
@@ -1207,7 +1207,7 @@ public class QueueManagerImpl implements QueueManager {
   protected void recoverAllInterestTypes(final Connection recoveredConnection,
       boolean isFirstNewConnection) {
     if (PoolImpl.BEFORE_RECOVER_INTEREST_CALLBACK_FLAG) {
-      BridgeObserver bo = BridgeObserverHolder.getInstance();
+      ClientServerObserver bo = ClientServerObserverHolder.getInstance();
       bo.beforeInterestRecovery();
     }
     recoverInterestList(recoveredConnection, false, true, isFirstNewConnection);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueStateImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueStateImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueStateImpl.java
index 9a6b3db..9f15155 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueStateImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/QueueStateImpl.java
@@ -24,8 +24,8 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl.PoolTask;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
@@ -257,7 +257,7 @@ public class QueueStateImpl implements QueueState {
         return;
       }
       if (PoolImpl.BEFORE_SENDING_CLIENT_ACK_CALLBACK_FLAG) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.beforeSendingClientAck();
       }
       //if ((qManager.getPool().getSubscriptionRedundancy() != 0) || (qManager.getPool().isDurableClient())) {



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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/doc-files/properties.html
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/doc-files/properties.html b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/doc-files/properties.html
index 6a335a8..0dd2bae 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/doc-files/properties.html
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/doc-files/properties.html
@@ -3590,9 +3590,7 @@ TBA Is this class even used?
 <p>
 <em>Boolean</em> (default is false)
 <p>
-See com.gemstone.gemfire.cache.util.BridgeLoader#init(Properties).
-<p>
-See com.gemstone.gemfire.cache.util.BridgeWriter#init(Properties).
+Removed in Geode 1.0 with removal of deprecated Bridge classes.
 <p>
 TBA 
 </dd>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java
index d4a33a6..1ef881c 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java
@@ -90,9 +90,7 @@ public class InternalFunctionService {
    */
   private static boolean isClientRegion(Region region) {
     LocalRegion localRegion = (LocalRegion)region;
-    return (localRegion.hasServerProxy()
-        || AbstractRegion.isBridgeLoader(localRegion.getCacheLoader()) || AbstractRegion
-        .isBridgeWriter(localRegion.getCacheWriter()));
+    return localRegion.hasServerProxy();
   }
 
   private static final FunctionServiceManager funcServiceManager = new FunctionServiceManager();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java
index f5cf679..e094648 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ha/HARegionQueue.java
@@ -68,7 +68,7 @@ import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedM
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.Conflatable;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -3013,9 +3013,9 @@ protected boolean checkEventForRemoval(Long counter, ThreadIdentifier threadid,
                 && !queueRemovalMessageList.isEmpty()) { // messages exist
               QueueRemovalMessage qrm = new QueueRemovalMessage();
               qrm.resetRecipients();
-              List<BridgeServerImpl> servers = this.cache.getBridgeServers();
+              List<CacheServerImpl> servers = this.cache.getCacheServers();
               List<DistributedMember> recipients = new LinkedList();
-              for (BridgeServerImpl server: servers) {
+              for (CacheServerImpl server: servers) {
                 recipients.addAll(server.getCacheServerAdvisor().adviseBridgeServers());
               }
               qrm.setRecipients(recipients);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionObserverHolder.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionObserverHolder.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionObserverHolder.java
index 0e8c3d8..0498328 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionObserverHolder.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionedRegionObserverHolder.java
@@ -44,7 +44,7 @@ public class PartitionedRegionObserverHolder {
     return oldObserver;
   }
 
-  /** Return the current BridgeObserver instance */
+  /** Return the current ClientServerObserver instance */
   public static final PartitionedRegionObserver getInstance()
   {
     return _instance;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/ConnectionProxy.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/ConnectionProxy.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/ConnectionProxy.java
index 2c7e0a5..685a142 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/ConnectionProxy.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/ConnectionProxy.java
@@ -7,20 +7,10 @@
  */
 package com.gemstone.gemfire.internal.cache.tier;
 
-import java.util.*;
-
-import com.gemstone.gemfire.CancelCriterion;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.util.EndpointDoesNotExistException;
-import com.gemstone.gemfire.cache.util.EndpointExistsException;
-import com.gemstone.gemfire.cache.util.EndpointInUseException;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.EventID;
-import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 
 /**
- * Defines the connection proxy interface, implementations of which
- * provide connection management facilities to the bridge loader. 
+ * Provides the version of the client.
  *
  * @author Sudhir Menon
  * @since 2.0.2
@@ -33,152 +23,4 @@ public interface ConnectionProxy {
    * @since 5.7
    */
   public static final Version VERSION = Version.CURRENT.getGemFireVersion();
-
-  public abstract void finalizeProxy();
-
-  /**
-   * Returns the load balancing policy in effect for this connection
-   * proxy.
-   */
-  public abstract String getLBPolicy();
-
-  /**
-   * Returns the number of milliseconds to wait before re-connecting
-   * to a dead server.
-   */
-  public abstract int getRetryInterval();
-
-  /**
-   * Returns the number of milliseconds to wait before timing out
-   * client/server communication.
-   */
-  public abstract int getReadTimeout();
-
-
-  /**
-   * Closes this connection proxy and all of its connections
-   */
-  public abstract void close();
-
-  /**
-   * Returned true if this ConnectionProxy has been initialized and not closed.
-   */
-  public abstract boolean isOpen();
-
-  /**
-   * Update bookkeeping on this proxy associated with the loss of a region.
-   * In particular, remove all region interests.
-   */
-  public abstract void detachRegion(Region r);
-
-  /**
-   * Returns the number of connections that should be created
-   * to every cache server.
-   */
-  public abstract int getConnectionsPerServer();
-
-  /**
-   * Notes that the server with the given name is unavailable
-   */
-  public abstract void setServerUnavailable(String name);
-
-  /**
-   * Notes that the server with the given name is available
-   */
-  public abstract void setServerAvailable(String name);
-
-  /**
-   * Stops this connection proxy and
-   */
-  public abstract void terminate();
-
-  /**
-   * Releases the connection associated with the current thread
-   */
-  public abstract void release();
-
-  /**
-   * Returns value of establishCallbackConnection property.
-   * @since 4.2.3
-   */
-  public boolean getEstablishCallbackConnection();
-
-  /**
-   * Add an <code>Endpoint</code> to the known <code>Endpoint</code>s.
-   *
-   * @param name The name of the endpoint to add
-   * @param host The host name or ip address of the endpoint to add
-   * @param port The port of the endpoint to add
-   *
-   * @throws EndpointExistsException if the <code>Endpoint</code> to be
-   * added already exists.
-   *
-   * @since 5.0.2
-   */
-  public void addEndpoint(String name, String host, int port)
-  throws EndpointExistsException;
-
-  /**
-   * Remove an <code>Endpoint</code> from the dead <code>Endpoint</code>s.
-   * The specified <code>Endpoint</code> must be dead.
-   *
-   * @param name The name of the endpoint to remove
-   * @param host The host name or ip address of the endpoint to remove
-   * @param port The port of the endpoint to remove
-   *
-   * @throws EndpointDoesNotExistException if the <code>Endpoint</code> to be
-   * removed doesn't exist.
-   *
-   * @throws EndpointInUseException if the <code>Endpoint</code> to be removed
-   * contains <code>Connection</code>s
-   *
-   * @since 5.0.2
-   */
-  public void removeEndpoint(String name, String host, int port)
-  throws EndpointDoesNotExistException, EndpointInUseException;
-
-  /**
-   * @return Returns the threadIdToSequenceId.
-   * @since 5.1
-   */
-  public Map getThreadIdToSequenceIdMap();
-
-  /**
-   * Verify if this EventId is already present in the map or not. If it is
-   * already present then return true
-   *
-   * @param eventId the EventId of the incoming event
-   * @return true if it is already present
-   * @since 5.1
-   */
-  public abstract boolean verifyIfDuplicate(EventID eventId, boolean addToMap);
-
-  /**
-   * @return Returns the redundancy number
-   * @since 5.1
-   */
-  public int getRedundancyLevel();
-
-  /**
-   * Returns the cancellation criterion for this proxy
-   * @return the cancellation criterion
-   */
-  public CancelCriterion getCancelCriterion();
-
-  /**
-   * The configurable expiry time of last received sequence ID
-   *
-   * @return The configurable expiry time of last received sequence ID
-   */
-  public long getMessageTrackingTimeout();
-
-  public boolean isDurableClient();
-
-  public void reuse();
-  public int getRetryAttempts();
-
-  /**
-   * Test hook for getting the client proxy membership id from this proxy.
-   */
-  public ClientProxyMembershipID getProxyID();
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/InternalBridgeMembership.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/InternalBridgeMembership.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/InternalBridgeMembership.java
deleted file mode 100755
index b0dc18f..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/InternalBridgeMembership.java
+++ /dev/null
@@ -1,715 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-package com.gemstone.gemfire.internal.cache.tier;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.CancelException;
-import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.util.BridgeMembershipEvent;
-import com.gemstone.gemfire.cache.util.BridgeMembershipListener;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl;
-import com.gemstone.gemfire.internal.cache.tier.sockets.ClientHealthMonitor;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
-import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
-import com.gemstone.gemfire.management.membership.ClientMembershipEvent;
-import com.gemstone.gemfire.management.membership.ClientMembershipListener;
-
-/**
- * Handles registration and event notification duties for
- * <code>BridgeMembershipListener</code>s. The public counterpart for this
- * class is {@link com.gemstone.gemfire.cache.util.BridgeMembership}.
- *
- * @author Kirk Lund
- * @since 4.2.1
- */
-public final class InternalBridgeMembership  {
-
-  private static final Logger logger = LogService.getLogger();
-  
-  /** 
-   * The membership listeners registered on this InternalBridgeMembership
-   * 
-   * This list is never modified in place, and a new list is installed
-   * only under the control of (@link #membershipLock}.
-   */
-  private static volatile List<BridgeMembershipListener> membershipListeners = Collections.emptyList();
-  
-  
-  private static volatile List<ClientMembershipListener> clientMembershipListeners = Collections.emptyList();
-  
-  /**
-   * Must be locked whenever references to the volatile field 
-   * {@link #membershipListeners} is changed.
-   */
-  private static final Object membershipLock = new Object();
-
-  /** 
-   * QueuedExecutor for firing BridgeMembershipEvents 
-   *
-   * Access synchronized via {@link #systems}
-   */
-  private static ThreadPoolExecutor executor;
-
-  private static final ThreadGroup threadGroup =
-      LoggingThreadGroup.createThreadGroup(
-          "BridgeMembership Event Invoker Group", logger);
-
-  /** List of connected <code>DistributedSystem</code>s */
-  private static final List systems = new ArrayList(1);
-
-  /**
-   * True if class is monitoring systems
-   * 
-   * @guarded.By InternalBridgeMembership.class
-   */
-  private static boolean isMonitoring = false;
-  
-  /**
-   * This work used to be in a class initializer.  Unfortunately, this allowed
-   * the class to escape before it was fully initialized, so now we just
-   * make sure this work is done before any public static method on it
-   * is invoked.
-   */
-  private static synchronized void startMonitoring() {
-    if (isMonitoring) {
-      return;
-    }
-    
-    synchronized(systems) {
-      // Initialize our own list of distributed systems via a connect listener
-      List existingSystems = InternalDistributedSystem.addConnectListener(
-        new InternalDistributedSystem.ConnectListener() {
-          public void onConnect(InternalDistributedSystem sys) {
-            addInternalDistributedSystem(sys);
-          }
-        });
-      
-      isMonitoring = true;
-      
-      // While still holding the lock on systems, add all currently known
-      // systems to our own list
-      for (Iterator iter = existingSystems.iterator(); iter.hasNext();) {
-        InternalDistributedSystem sys = (InternalDistributedSystem) iter.next();
-        try {
-          if (sys.isConnected()) {
-            addInternalDistributedSystem(sys);
-          }
-        }
-        catch (DistributedSystemDisconnectedException e) {
-          // it doesn't care (bug 37379)
-        }
-      }
-      
-    } // synchronized
-  }
-  
-  private InternalBridgeMembership() {}
-
-  /**
-   * Registers a {@link BridgeMembershipListener} for notification of
-   * connection changes for BridgeServers and bridge clients.
-   * @param listener a BridgeMembershipListener to be registered
-   * @deprecated use newer registerClientMembershipListener instead
-   */
-  public static void registerBridgeMembershipListener(BridgeMembershipListener listener) {
-    startMonitoring();
-    synchronized (membershipLock) {
-      List<BridgeMembershipListener> oldListeners = membershipListeners;
-      if (!oldListeners.contains(listener)) {
-        List<BridgeMembershipListener> newListeners = new ArrayList<BridgeMembershipListener>(oldListeners);
-        newListeners.add(listener);
-        membershipListeners = newListeners;
-      }
-    }
-  }
-  
-  /**
-   * Registers a {@link ClientMembershipListener} for notification of connection
-   * changes for CacheServer and clients.
-   * 
-   * @param listener
-   *          a ClientMembershipListener to be registered
-   */
-  public static void registerClientMembershipListener(ClientMembershipListener listener) {
-    startMonitoring();
-    synchronized (membershipLock) {
-      List<ClientMembershipListener> oldListeners = clientMembershipListeners;
-      if (!oldListeners.contains(listener)) {
-        List<ClientMembershipListener> newListeners = new ArrayList<ClientMembershipListener>(oldListeners);
-        newListeners.add(listener);
-        clientMembershipListeners = newListeners;
-      }
-    }
-  }
-  
-  /**
-   * Removes registration of a previously registered {@link
-   * BridgeMembershipListener}.
-   * @param listener a BridgeMembershipListener to be unregistered
-   * @deprecated
-   */
-  public static void unregisterBridgeMembershipListener(BridgeMembershipListener listener) {
-    startMonitoring();
-    synchronized (membershipLock) {
-      List<BridgeMembershipListener> oldListeners = membershipListeners;
-      if (oldListeners.contains(listener)) {
-        List<BridgeMembershipListener> newListeners = new ArrayList<BridgeMembershipListener>(oldListeners);
-        if (newListeners.remove(listener)) {
-          membershipListeners = newListeners;
-        }
-      }
-    }
-  }
-  
-  /**
-   * Removes registration of a previously registered
-   * {@link ClientMembershipListener}.
-   * 
-   * @param listener
-   *          a ClientMembershipListener to be unregistered
-   */
-  public static void unregisterClientMembershipListener(ClientMembershipListener listener) {
-    startMonitoring();
-    synchronized (membershipLock) {
-      List<ClientMembershipListener> oldListeners = clientMembershipListeners;
-      if (oldListeners.contains(listener)) {
-        List<ClientMembershipListener> newListeners = new ArrayList<ClientMembershipListener>(oldListeners);
-        if (newListeners.remove(listener)) {
-          clientMembershipListeners = newListeners;
-        }
-      }
-    }
-  }
-
-  /**
-   * Returns an array of all the currently registered
-   * <code>BridgeMembershipListener</code>s. Modifications to the returned
-   * array will not effect the registration of these listeners.
-   * @return the registered <code>BridgeMembershipListener</code>s; an empty
-   * array if no listeners
-   * @deprecated
-   */
-  public static BridgeMembershipListener[] getBridgeMembershipListeners() {
-    startMonitoring();
-    // Synchronization is not needed because we never modify this list
-    // in place.
-    
-    List<BridgeMembershipListener> l = membershipListeners; // volatile fetch
-    // convert to an array
-    BridgeMembershipListener[] listeners = (BridgeMembershipListener[]) 
-        l.toArray(new BridgeMembershipListener[l.size()]);
-    return listeners;
-  }
-  
-  /**
-   * Returns an array of all the currently registered
-   * <code>ClientMembershipListener</code>s. Modifications to the returned array
-   * will not effect the registration of these listeners.
-   * 
-   * @return the registered <code>ClientMembershipListener</code>s; an empty
-   *         array if no listeners
-   */
-  public static ClientMembershipListener[] getClientMembershipListeners() {
-    startMonitoring();
-    // Synchronization is not needed because we never modify this list
-    // in place.
-
-    List<ClientMembershipListener> l = clientMembershipListeners; // volatile fetch
-    // convert to an array
-    ClientMembershipListener[] listeners = (ClientMembershipListener[]) l
-        .toArray(new ClientMembershipListener[l.size()]);
-    return listeners;
-  }
-
-  /**
-   * Removes registration of all currently registered
-   * <code>BridgeMembershipListener<code>s. and <code>ClientMembershipListener<code>s.
-   */
-  public static void unregisterAllListeners() {
-    startMonitoring();
-    synchronized (membershipLock) {
-      membershipListeners = new ArrayList<BridgeMembershipListener>();
-      clientMembershipListeners = new ArrayList<ClientMembershipListener>();
-    }
-  }
-  
-  
-  
-  /**
-   * Returns a map of client memberIds to count of connections to that client.
-   * The map entry key is a String representation of the client memberId, and
-   * the map entry value is an Integer count of connections to that client.
-   * Since a single client can have multiple ConnectionProxy objects, this 
-   * map will contain all the Connection objects across the ConnectionProxies
-   * @param onlyClientsNotifiedByThisServer true will return only those clients
-   * that are actively being updated by this server
-   * @return map of client memberIds to count of connections to that client
-   * 
-   * 
-   */
-  public static Map getConnectedClients(boolean onlyClientsNotifiedByThisServer) {
-    ClientHealthMonitor chMon = ClientHealthMonitor.getInstance();
-    Set filterProxyIDs = null;
-    if(onlyClientsNotifiedByThisServer) {
-      // Note it is not necessary to synchronize on the list of bridge servers here, 
-      // since this is only a status (snapshot) of the system.
-      for (Iterator bsii = CacheFactory.getAnyInstance().getBridgeServers().iterator(); bsii.hasNext(); ) {
-        BridgeServerImpl bsi = (BridgeServerImpl) bsii.next();
-        AcceptorImpl ai = bsi.getAcceptor();
-        if (ai != null && ai.getCacheClientNotifier() != null) {
-          if (filterProxyIDs != null) {
-            // notifierClients is a copy set from CacheClientNotifier
-            filterProxyIDs.addAll(ai.getCacheClientNotifier().getActiveClients());
-          }
-          else {
-            // notifierClients is a copy set from CacheClientNotifier
-            filterProxyIDs = ai.getCacheClientNotifier().getActiveClients();
-          }
-        }
-      }
-    }
-
-    Map map = chMon.getConnectedClients(filterProxyIDs);
-   /*if (onlyClientsNotifiedByThisServer) {
-      Map notifyMap = new HashMap();
-      
-      for (Iterator iter = map.keySet().iterator(); iter.hasNext();) {
-        String memberId = (String) iter.next();
-        if (notifierClients.contains(memberId)) {
-          // found memberId that is notified by this server
-          notifyMap.put(memberId, map.get(memberId));
-        }
-      }
-      map = notifyMap;
-    }*/
-    return map;
-  }
-  
-  /**
-   * This method returns the CacheClientStatus for all the clients that are
-   * connected to this server. This method returns all clients irrespective of
-   * whether subscription is enabled or not. 
-   * 
-   * @return Map of ClientProxyMembershipID against CacheClientStatus objects.
-   */
-  public static Map getStatusForAllClientsIgnoreSubscriptionStatus() {
-    Map result = new HashMap();
-    if (ClientHealthMonitor.getInstance() != null)
-      result = ClientHealthMonitor.getInstance().getStatusForAllClients();
-
-    return result;
-  }  
-
-  /**
-   * Caller must synchronize on cache.allBridgeServersLock
-   * @return all the clients
-   */
-  public static Map getConnectedClients() {
-
-    // Get all clients
-    Map allClients = new HashMap();
-    for (Iterator bsii = CacheFactory.getAnyInstance().getBridgeServers().iterator(); bsii.hasNext(); ) {
-      BridgeServerImpl bsi = (BridgeServerImpl) bsii.next();
-      AcceptorImpl ai = bsi.getAcceptor();
-      if (ai != null && ai.getCacheClientNotifier() != null) {
-        allClients.putAll(ai.getCacheClientNotifier().getAllClients());
-      }
-    }
-
-    // Fill in the missing info, if HealthMonitor started
-    if (ClientHealthMonitor.getInstance()!=null)
-        ClientHealthMonitor.getInstance().fillInClientInfo(allClients);
-
-    return allClients;
-  }
-
-  public static Map getClientQueueSizes() {
-    Map clientQueueSizes = new HashMap();
-    GemFireCacheImpl c =  (GemFireCacheImpl)CacheFactory.getAnyInstance();
-    if (c==null) // Add a NULL Check
-      return clientQueueSizes;
-
-    for (Iterator bsii = c.getBridgeServers().iterator(); bsii.hasNext(); ) {
-      BridgeServerImpl bsi = (BridgeServerImpl) bsii.next();
-      AcceptorImpl ai = bsi.getAcceptor();
-      if (ai != null && ai.getCacheClientNotifier() != null) {
-        clientQueueSizes.putAll(ai.getCacheClientNotifier().getClientQueueSizes());
-      }
-    } // for
-    return clientQueueSizes;
-  }
-
-  /**
-   * Returns a map of servers to count of pools connected to that server.
-   * The map entry key is a String representation of the server, 
-   * @return map of servers to count of pools using that server
-   */
-  public static Map getConnectedServers() {
-    final Map map = new HashMap(); // KEY:server (String), VALUE:List of active endpoints
-    // returns an unmodifiable set
-    Map/*<String,Pool>*/ poolMap = PoolManager.getAll();
-    Iterator pools = poolMap.values().iterator();
-    while(pools.hasNext()) {
-      PoolImpl pi = (PoolImpl)pools.next();
-      Map/*<ServerLocation,Endpoint>*/ eps = pi.getEndpointMap();
-      Iterator it = eps.entrySet().iterator();
-      while(it.hasNext()) {
-        Map.Entry entry = (Map.Entry)it.next();
-        ServerLocation loc = (ServerLocation)entry.getKey();
-        com.gemstone.gemfire.cache.client.internal.Endpoint ep = (com.gemstone.gemfire.cache.client.internal.Endpoint)entry.getValue();
-        String server = loc.getHostName()+"["+loc.getPort()+"]";
-        Integer count = (Integer)map.get(server);
-        if(count==null) {
-          map.put(server,Integer.valueOf(1));  
-        } else {
-          map.put(server,Integer.valueOf(count.intValue()+1));
-        }
-      }
-    }
-    return map;
-  }
-
-  public static Map getConnectedIncomingGateways() {
-    Map connectedIncomingGateways = null;
-    ClientHealthMonitor chMon = ClientHealthMonitor.getInstance();
-    if (chMon == null) {
-      connectedIncomingGateways = new HashMap();
-    } else {
-      connectedIncomingGateways = chMon.getConnectedIncomingGateways();
-    }
-    return connectedIncomingGateways;
-  }
-  
-  
-
-  /**
-   * Notifies registered listeners that a bridge member has joined. The new
-   * member may be a bridge client connecting to this process or a bridge
-   * server that this process has just connected to.
-   *
-   * @param member the <code>DistributedMember</code>
-   * @param client true if the member is a bridge client; false if bridge server
-   */
-  public static void notifyJoined(final DistributedMember member, final boolean client) {
-    startMonitoring();
-    ThreadPoolExecutor queuedExecutor = executor;
-    if (queuedExecutor == null) {
-      return;
-    }
-
-    final BridgeMembershipEvent event =
-        new InternalBridgeMembershipEvent(member, client);
-    if (forceSynchronous) {
-      doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_JOINED);
-      doNotifyBridgeMembershipListener(member, client, event,EventType.CLIENT_JOINED);
-    }
-    else {
-      try {
-          queuedExecutor.execute(new Runnable() {
-              public void run() {
-                doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_JOINED);
-                doNotifyBridgeMembershipListener(member, client, event,EventType.CLIENT_JOINED);
-              }
-            });
-      }
-      catch (RejectedExecutionException e) {
-        // executor must have been shutdown
-        }
-    }
-  }
-
-
-
-  /**
-   * Notifies registered listeners that a bridge member has left. The departed
-   * member may be a bridge client previously connected to this process or a
-   * bridge server that this process was connected to.
-   *
-   * @param member the <code>DistributedMember</code>
-   * @param client true if the member is a bridge client; false if bridge server
-   */
-  public static void notifyLeft(final DistributedMember member, final boolean client) {
-    startMonitoring();
-    ThreadPoolExecutor queuedExecutor = executor;
-    if (queuedExecutor == null) {
-      return;
-    }
-
-    
-    final BridgeMembershipEvent event =
-        new InternalBridgeMembershipEvent(member, client);
-    if (forceSynchronous) {
-      doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_LEFT);
-      doNotifyBridgeMembershipListener(member, client, event,EventType.CLIENT_LEFT);
-    }
-    else {
-      try {
-          queuedExecutor.execute(new Runnable() {
-              public void run() {
-                doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_LEFT);
-                doNotifyBridgeMembershipListener(member, client, event,EventType.CLIENT_LEFT);
-              }
-            });
-      }
-      catch (RejectedExecutionException e) {
-        // executor must have been shutdown
-        }
-    }
-  }
-
-
-  /**
-   * Notifies registered listeners that a bridge member has crashed. The
-   * departed member may be a bridge client previously connected to this
-   * process or a bridge server that this process was connected to.
-   *
-   * @param member the <code>DistributedMember</code>
-   * @param client true if the member is a bridge client; false if bridge server
-   */
-  public static void notifyCrashed(final DistributedMember member, final boolean client) {
-    ThreadPoolExecutor queuedExecutor = executor;
-    if (queuedExecutor == null) {
-      return;
-    }
-
-    final BridgeMembershipEvent event =
-        new InternalBridgeMembershipEvent(member, client);
-    if (forceSynchronous) {
-      doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_CRASHED);
-      doNotifyBridgeMembershipListener(member, client, event,EventType.CLIENT_CRASHED);
-    }
-    else {
-
-      try {
-          queuedExecutor.execute(new Runnable() {
-            public void run() {
-              doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_CRASHED);
-              doNotifyBridgeMembershipListener(member, client, event,EventType.CLIENT_CRASHED);
-            }
-          });
-      }
-      catch (RejectedExecutionException e) {
-        // executor must have been shutdown
-        }
-    }
-  }
-
-  private static void doNotifyClientMembershipListener(DistributedMember member, boolean client,
-      ClientMembershipEvent clientMembershipEvent, EventType eventType) {
-
-    for (Iterator<ClientMembershipListener> iter = clientMembershipListeners.iterator(); iter.hasNext();) {
-
-      ClientMembershipListener listener = iter.next();
-      try {
-        if (eventType.equals(EventType.CLIENT_JOINED)) {
-          listener.memberJoined(clientMembershipEvent);
-        } else if (eventType.equals(EventType.CLIENT_LEFT)) {
-          listener.memberLeft(clientMembershipEvent);
-        } else {
-          listener.memberCrashed(clientMembershipEvent);
-        }
-      } catch (CancelException e) {
-        // this can be thrown by a bridge server when the system is shutting
-        // down
-        return;
-      } catch (VirtualMachineError e) {
-        SystemFailure.initiateFailure(e);
-        throw e;
-      } catch (Throwable t) {
-        SystemFailure.checkFailure();
-        logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_UNEXPECTED_EXCEPTION), t);
-      }
-    }
-  }
-  
-  private static void doNotifyBridgeMembershipListener(DistributedMember member, boolean client,
-      BridgeMembershipEvent bridgeMembershipEvent, EventType eventType) {
-
-    for (Iterator<BridgeMembershipListener> iter = membershipListeners.iterator(); iter.hasNext();) {
-
-      BridgeMembershipListener listener = iter.next();
-      try {
-        if (eventType.equals(EventType.CLIENT_JOINED)) {
-          listener.memberJoined(bridgeMembershipEvent);
-        } else if (eventType.equals(EventType.CLIENT_LEFT)) {
-          listener.memberLeft(bridgeMembershipEvent);
-        } else {
-          listener.memberCrashed(bridgeMembershipEvent);
-        }
-      } catch (CancelException e) {
-        // this can be thrown by a bridge server when the system is shutting
-        // down
-        return;
-      } catch (VirtualMachineError e) {
-        SystemFailure.initiateFailure(e);
-        throw e;
-      } catch (Throwable t) {
-        SystemFailure.checkFailure();
-        logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_UNEXPECTED_EXCEPTION), t);
-      }
-    }
-  }
-  
-
-//  /**
-//   * Returns true if there are any registered
-//   * <code>BridgeMembershipListener</code>s.
-//   */
-//  private static boolean hasBridgeMembershipListeners() {
-//    synchronized (membershipLock) {
-//      return !membershipListeners.isEmpty();
-//    }
-//  }
-
-  protected static void addInternalDistributedSystem(InternalDistributedSystem s) {
-    synchronized(systems) {
-      s.addDisconnectListener(
-        new InternalDistributedSystem.DisconnectListener() {
-          @Override
-          public String toString() {
-            return "Disconnect listener for InternalBridgeMembership";
-          }
-          
-          public void onDisconnect(InternalDistributedSystem ss) {
-            removeInternalDistributedSystem(ss);
-          }
-        });
-      systems.add(s);
-      // make sure executor is alive
-      ensureExecutorIsRunning(); // optimized to do nothing if already running
-    }
-  }
-
-  protected static void removeInternalDistributedSystem(InternalDistributedSystem sys) {
-    synchronized(systems) {
-      systems.remove(sys);
-      if (systems.isEmpty()) {
-        // clean up executor
-/*
-Object[] queueElementsBefore = new Object[executorQueue.size()];
-queueElementsBefore = executorQueue.toArray(queueElementsBefore);
-System.out.println("Before shut down, the executor's queue contains the following " + queueElementsBefore.length + " elements");
-for (int i=0; i<queueElementsBefore.length; i++) {
-  System.out.println("\t" + queueElementsBefore[i]);
-}
-*/
-        if (executor != null) {
-          executor.shutdown();
-        }
-/*
-Object[] queueElementsAfter = new Object[executorQueue.size()];
-queueElementsAfter = executorQueue.toArray(queueElementsAfter);
-System.out.println("After shut down, the executor's queue contains the following " + queueElementsAfter.length + " elements");
-for (int i=0; i<queueElementsAfter.length; i++) {
-  System.out.println("\t" + queueElementsAfter[i]);
-}
-*/
-        // deadcoded this clear to fix bug 35675 - clearing removed the shutdown token from the queue!
-        // executorQueue.clear();
-        executor = null;
-      }
-    }
-  }
-
-  /**
-   * @guarded.By {@link #systems}
-   */
-  private static void ensureExecutorIsRunning() {
-    // protected by calling method synchronized on systems
-    if (executor == null) {
-      final ThreadGroup group = threadGroup;
-      ThreadFactory tf = new ThreadFactory() {
-          public Thread newThread(Runnable command) {
-            Thread thread =
-                new Thread(group, command, "BridgeMembership Event Invoker");
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-      LinkedBlockingQueue q = new LinkedBlockingQueue();
-      executor = new ThreadPoolExecutor(1, 1/*max unused*/,
-                                        15, TimeUnit.SECONDS, q, tf);
-    }
-  }
-
-  /**
-   * Internal implementation of BridgeMembershipEvent.
-   */
-  protected static class InternalBridgeMembershipEvent
-  implements BridgeMembershipEvent,ClientMembershipEvent {
-
-    private final DistributedMember member;
-    private final boolean client;
-
-    /** Constructs new instance of event */
-    protected InternalBridgeMembershipEvent(DistributedMember member, boolean client) {
-      this.member = member;
-      this.client = client;
-    }
-
-    public DistributedMember getMember() {
-      return this.member;
-    }
-
-    public String getMemberId() {
-      return this.member == null ? "unknown" : this.member.getId();
-    }
-
-    public boolean isClient() {
-      return this.client;
-    }
-
-    @Override // GemStoneAddition
-    public String toString() {
-      final StringBuffer sb = new StringBuffer("[BridgeMembershipEvent: ");
-      sb.append("member=").append(this.member);
-      sb.append(", isClient=").append(this.client);
-      sb.append("]");
-      return sb.toString();
-    }
-  }
-  
-  /** If set to true for testing then notification will be synchronous */
-  private static boolean forceSynchronous = false;
-  /** Set to true if synchronous notification is needed for testing */
-  public static void setForceSynchronous(boolean value) {
-    forceSynchronous = value;
-  }
-  
-  private static enum EventType{
-    CLIENT_JOINED,
-    CLIENT_LEFT,
-    CLIENT_CRASHED
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/InternalClientMembership.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/InternalClientMembership.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/InternalClientMembership.java
new file mode 100755
index 0000000..22fa0e3
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/InternalClientMembership.java
@@ -0,0 +1,617 @@
+/*=========================================================================
+ * Copyright (c) 2002-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
+ * more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.internal.cache.tier;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.client.PoolManager;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.ServerLocation;
+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.ClientHealthMonitor;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.logging.LoggingThreadGroup;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
+import com.gemstone.gemfire.management.membership.ClientMembershipEvent;
+import com.gemstone.gemfire.management.membership.ClientMembershipListener;
+
+/**
+ * Handles registration and event notification duties for
+ * <code>ClientMembershipListener</code>s. The public counterpart for this
+ * class is {@link com.gemstone.gemfire.management.membership.ClientMembership}.
+ *
+ * @author Kirk Lund
+ * @since 4.2.1
+ */
+public final class InternalClientMembership  {
+
+  private static final Logger logger = LogService.getLogger();
+  
+  /** 
+   * The membership listeners registered on this InternalClientMembership
+   * 
+   * This list is never modified in place, and a new list is installed
+   * only under the control of (@link #membershipLock}.
+   */
+  private static volatile List<ClientMembershipListener> clientMembershipListeners = Collections.emptyList();
+  
+  /**
+   * Must be locked whenever references to the volatile field 
+   * {@link #membershipListeners} is changed.
+   */
+  private static final Object membershipLock = new Object();
+
+  /** 
+   * QueuedExecutor for firing ClientMembershipEvents 
+   *
+   * Access synchronized via {@link #systems}
+   */
+  private static ThreadPoolExecutor executor;
+
+  private static final ThreadGroup threadGroup =
+      LoggingThreadGroup.createThreadGroup(
+          "ClientMembership Event Invoker Group", logger);
+
+  /** List of connected <code>DistributedSystem</code>s */
+  private static final List systems = new ArrayList(1);
+
+  /**
+   * True if class is monitoring systems
+   * 
+   * @guarded.By InternalClientMembership.class
+   */
+  private static boolean isMonitoring = false;
+  
+  /**
+   * This work used to be in a class initializer.  Unfortunately, this allowed
+   * the class to escape before it was fully initialized, so now we just
+   * make sure this work is done before any public static method on it
+   * is invoked.
+   */
+  private static synchronized void startMonitoring() {
+    if (isMonitoring) {
+      return;
+    }
+    
+    synchronized(systems) {
+      // Initialize our own list of distributed systems via a connect listener
+      List existingSystems = InternalDistributedSystem.addConnectListener(
+        new InternalDistributedSystem.ConnectListener() {
+          public void onConnect(InternalDistributedSystem sys) {
+            addInternalDistributedSystem(sys);
+          }
+        });
+      
+      isMonitoring = true;
+      
+      // While still holding the lock on systems, add all currently known
+      // systems to our own list
+      for (Iterator iter = existingSystems.iterator(); iter.hasNext();) {
+        InternalDistributedSystem sys = (InternalDistributedSystem) iter.next();
+        try {
+          if (sys.isConnected()) {
+            addInternalDistributedSystem(sys);
+          }
+        }
+        catch (DistributedSystemDisconnectedException e) {
+          // it doesn't care (bug 37379)
+        }
+      }
+      
+    } // synchronized
+  }
+  
+  private InternalClientMembership() {}
+
+  /**
+   * Registers a {@link ClientMembershipListener} for notification of connection
+   * changes for CacheServer and clients.
+   * 
+   * @param listener
+   *          a ClientMembershipListener to be registered
+   */
+  public static void registerClientMembershipListener(ClientMembershipListener listener) {
+    startMonitoring();
+    synchronized (membershipLock) {
+      List<ClientMembershipListener> oldListeners = clientMembershipListeners;
+      if (!oldListeners.contains(listener)) {
+        List<ClientMembershipListener> newListeners = new ArrayList<ClientMembershipListener>(oldListeners);
+        newListeners.add(listener);
+        clientMembershipListeners = newListeners;
+      }
+    }
+  }
+  
+  /**
+   * Removes registration of a previously registered
+   * {@link ClientMembershipListener}.
+   * 
+   * @param listener
+   *          a ClientMembershipListener to be unregistered
+   */
+  public static void unregisterClientMembershipListener(ClientMembershipListener listener) {
+    startMonitoring();
+    synchronized (membershipLock) {
+      List<ClientMembershipListener> oldListeners = clientMembershipListeners;
+      if (oldListeners.contains(listener)) {
+        List<ClientMembershipListener> newListeners = new ArrayList<ClientMembershipListener>(oldListeners);
+        if (newListeners.remove(listener)) {
+          clientMembershipListeners = newListeners;
+        }
+      }
+    }
+  }
+
+  /**
+   * Returns an array of all the currently registered
+   * <code>ClientMembershipListener</code>s. Modifications to the returned array
+   * will not effect the registration of these listeners.
+   * 
+   * @return the registered <code>ClientMembershipListener</code>s; an empty
+   *         array if no listeners
+   */
+  public static ClientMembershipListener[] getClientMembershipListeners() {
+    startMonitoring();
+    // Synchronization is not needed because we never modify this list
+    // in place.
+
+    List<ClientMembershipListener> l = clientMembershipListeners; // volatile fetch
+    // convert to an array
+    ClientMembershipListener[] listeners = (ClientMembershipListener[]) l
+        .toArray(new ClientMembershipListener[l.size()]);
+    return listeners;
+  }
+
+  /**
+   * Removes registration of all currently registered
+   * <code>ClientMembershipListener<code>s. and <code>ClientMembershipListener<code>s.
+   */
+  public static void unregisterAllListeners() {
+    startMonitoring();
+    synchronized (membershipLock) {
+      clientMembershipListeners = new ArrayList<ClientMembershipListener>();
+    }
+  }
+  
+  
+  
+  /**
+   * Returns a map of client memberIds to count of connections to that client.
+   * The map entry key is a String representation of the client memberId, and
+   * the map entry value is an Integer count of connections to that client.
+   * Since a single client can have multiple ConnectionProxy objects, this 
+   * map will contain all the Connection objects across the ConnectionProxies
+   * @param onlyClientsNotifiedByThisServer true will return only those clients
+   * that are actively being updated by this server
+   * @return map of client memberIds to count of connections to that client
+   * 
+   * 
+   */
+  public static Map getConnectedClients(boolean onlyClientsNotifiedByThisServer) {
+    ClientHealthMonitor chMon = ClientHealthMonitor.getInstance();
+    Set filterProxyIDs = null;
+    if(onlyClientsNotifiedByThisServer) {
+      // Note it is not necessary to synchronize on the list of Client servers here, 
+      // since this is only a status (snapshot) of the system.
+      for (Iterator bsii = CacheFactory.getAnyInstance().getCacheServers().iterator(); bsii.hasNext(); ) {
+        CacheServerImpl bsi = (CacheServerImpl) bsii.next();
+        AcceptorImpl ai = bsi.getAcceptor();
+        if (ai != null && ai.getCacheClientNotifier() != null) {
+          if (filterProxyIDs != null) {
+            // notifierClients is a copy set from CacheClientNotifier
+            filterProxyIDs.addAll(ai.getCacheClientNotifier().getActiveClients());
+          }
+          else {
+            // notifierClients is a copy set from CacheClientNotifier
+            filterProxyIDs = ai.getCacheClientNotifier().getActiveClients();
+          }
+        }
+      }
+    }
+
+    Map map = chMon.getConnectedClients(filterProxyIDs);
+   /*if (onlyClientsNotifiedByThisServer) {
+      Map notifyMap = new HashMap();
+      
+      for (Iterator iter = map.keySet().iterator(); iter.hasNext();) {
+        String memberId = (String) iter.next();
+        if (notifierClients.contains(memberId)) {
+          // found memberId that is notified by this server
+          notifyMap.put(memberId, map.get(memberId));
+        }
+      }
+      map = notifyMap;
+    }*/
+    return map;
+  }
+  
+  /**
+   * This method returns the CacheClientStatus for all the clients that are
+   * connected to this server. This method returns all clients irrespective of
+   * whether subscription is enabled or not. 
+   * 
+   * @return Map of ClientProxyMembershipID against CacheClientStatus objects.
+   */
+  public static Map getStatusForAllClientsIgnoreSubscriptionStatus() {
+    Map result = new HashMap();
+    if (ClientHealthMonitor.getInstance() != null)
+      result = ClientHealthMonitor.getInstance().getStatusForAllClients();
+
+    return result;
+  }  
+
+  /**
+   * Caller must synchronize on cache.allClientServersLock
+   * @return all the clients
+   */
+  public static Map getConnectedClients() {
+
+    // Get all clients
+    Map allClients = new HashMap();
+    for (Iterator bsii = CacheFactory.getAnyInstance().getCacheServers().iterator(); bsii.hasNext(); ) {
+      CacheServerImpl bsi = (CacheServerImpl) bsii.next();
+      AcceptorImpl ai = bsi.getAcceptor();
+      if (ai != null && ai.getCacheClientNotifier() != null) {
+        allClients.putAll(ai.getCacheClientNotifier().getAllClients());
+      }
+    }
+
+    // Fill in the missing info, if HealthMonitor started
+    if (ClientHealthMonitor.getInstance()!=null)
+        ClientHealthMonitor.getInstance().fillInClientInfo(allClients);
+
+    return allClients;
+  }
+
+  public static Map getClientQueueSizes() {
+    Map clientQueueSizes = new HashMap();
+    GemFireCacheImpl c =  (GemFireCacheImpl)CacheFactory.getAnyInstance();
+    if (c==null) // Add a NULL Check
+      return clientQueueSizes;
+
+    for (Iterator bsii = c.getCacheServers().iterator(); bsii.hasNext(); ) {
+      CacheServerImpl bsi = (CacheServerImpl) bsii.next();
+      AcceptorImpl ai = bsi.getAcceptor();
+      if (ai != null && ai.getCacheClientNotifier() != null) {
+        clientQueueSizes.putAll(ai.getCacheClientNotifier().getClientQueueSizes());
+      }
+    } // for
+    return clientQueueSizes;
+  }
+
+  /**
+   * Returns a map of servers to count of pools connected to that server.
+   * The map entry key is a String representation of the server, 
+   * @return map of servers to count of pools using that server
+   */
+  public static Map getConnectedServers() {
+    final Map map = new HashMap(); // KEY:server (String), VALUE:List of active endpoints
+    // returns an unmodifiable set
+    Map/*<String,Pool>*/ poolMap = PoolManager.getAll();
+    Iterator pools = poolMap.values().iterator();
+    while(pools.hasNext()) {
+      PoolImpl pi = (PoolImpl)pools.next();
+      Map/*<ServerLocation,Endpoint>*/ eps = pi.getEndpointMap();
+      Iterator it = eps.entrySet().iterator();
+      while(it.hasNext()) {
+        Map.Entry entry = (Map.Entry)it.next();
+        ServerLocation loc = (ServerLocation)entry.getKey();
+        com.gemstone.gemfire.cache.client.internal.Endpoint ep = (com.gemstone.gemfire.cache.client.internal.Endpoint)entry.getValue();
+        String server = loc.getHostName()+"["+loc.getPort()+"]";
+        Integer count = (Integer)map.get(server);
+        if(count==null) {
+          map.put(server,Integer.valueOf(1));  
+        } else {
+          map.put(server,Integer.valueOf(count.intValue()+1));
+        }
+      }
+    }
+    return map;
+  }
+
+  public static Map getConnectedIncomingGateways() {
+    Map connectedIncomingGateways = null;
+    ClientHealthMonitor chMon = ClientHealthMonitor.getInstance();
+    if (chMon == null) {
+      connectedIncomingGateways = new HashMap();
+    } else {
+      connectedIncomingGateways = chMon.getConnectedIncomingGateways();
+    }
+    return connectedIncomingGateways;
+  }
+  
+  
+
+  /**
+   * Notifies registered listeners that a Client member has joined. The new
+   * member may be a client connecting to this process or a
+   * server that this process has just connected to.
+   *
+   * @param member the <code>DistributedMember</code>
+   * @param client true if the member is a client; false if server
+   */
+  public static void notifyJoined(final DistributedMember member, final boolean client) {
+    startMonitoring();
+    ThreadPoolExecutor queuedExecutor = executor;
+    if (queuedExecutor == null) {
+      return;
+    }
+
+    final ClientMembershipEvent event =
+        new InternalClientMembershipEvent(member, client);
+    if (forceSynchronous) {
+      doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_JOINED);
+    }
+    else {
+      try {
+          queuedExecutor.execute(new Runnable() {
+              public void run() {
+                doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_JOINED);
+              }
+            });
+      }
+      catch (RejectedExecutionException e) {
+        // executor must have been shutdown
+        }
+    }
+  }
+
+
+
+  /**
+   * Notifies registered listeners that a member has left. The departed
+   * member may be a client previously connected to this process or a
+   * server that this process was connected to.
+   *
+   * @param member the <code>DistributedMember</code>
+   * @param client true if the member is a client; false if server
+   */
+  public static void notifyLeft(final DistributedMember member, final boolean client) {
+    startMonitoring();
+    ThreadPoolExecutor queuedExecutor = executor;
+    if (queuedExecutor == null) {
+      return;
+    }
+
+    
+    final ClientMembershipEvent event =
+        new InternalClientMembershipEvent(member, client);
+    if (forceSynchronous) {
+      doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_LEFT);
+    }
+    else {
+      try {
+          queuedExecutor.execute(new Runnable() {
+              public void run() {
+                doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_LEFT);
+              }
+            });
+      }
+      catch (RejectedExecutionException e) {
+        // executor must have been shutdown
+        }
+    }
+  }
+
+
+  /**
+   * Notifies registered listeners that a member has crashed. The
+   * departed member may be a client previously connected to this
+   * process or a server that this process was connected to.
+   *
+   * @param member the <code>DistributedMember</code>
+   * @param client true if the member is a client; false if server
+   */
+  public static void notifyCrashed(final DistributedMember member, final boolean client) {
+    ThreadPoolExecutor queuedExecutor = executor;
+    if (queuedExecutor == null) {
+      return;
+    }
+
+    final ClientMembershipEvent event =
+        new InternalClientMembershipEvent(member, client);
+    if (forceSynchronous) {
+      doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_CRASHED);
+    }
+    else {
+
+      try {
+          queuedExecutor.execute(new Runnable() {
+            public void run() {
+              doNotifyClientMembershipListener(member, client, event,EventType.CLIENT_CRASHED);
+            }
+          });
+      }
+      catch (RejectedExecutionException e) {
+        // executor must have been shutdown
+        }
+    }
+  }
+
+  private static void doNotifyClientMembershipListener(DistributedMember member, boolean client,
+      ClientMembershipEvent clientMembershipEvent, EventType eventType) {
+
+    for (Iterator<ClientMembershipListener> iter = clientMembershipListeners.iterator(); iter.hasNext();) {
+
+      ClientMembershipListener listener = iter.next();
+      try {
+        if (eventType.equals(EventType.CLIENT_JOINED)) {
+          listener.memberJoined(clientMembershipEvent);
+        } else if (eventType.equals(EventType.CLIENT_LEFT)) {
+          listener.memberLeft(clientMembershipEvent);
+        } else {
+          listener.memberCrashed(clientMembershipEvent);
+        }
+      } catch (CancelException e) {
+        // this can be thrown by a server when the system is shutting
+        // down
+        return;
+      } catch (VirtualMachineError e) {
+        SystemFailure.initiateFailure(e);
+        throw e;
+      } catch (Throwable t) {
+        SystemFailure.checkFailure();
+        logger.warn(LocalizedMessage.create(LocalizedStrings.LocalRegion_UNEXPECTED_EXCEPTION), t);
+      }
+    }
+  }
+  
+//  /**
+//   * Returns true if there are any registered
+//   * <code>ClientMembershipListener</code>s.
+//   */
+//  private static boolean hasClientMembershipListeners() {
+//    synchronized (membershipLock) {
+//      return !membershipListeners.isEmpty();
+//    }
+//  }
+
+  protected static void addInternalDistributedSystem(InternalDistributedSystem s) {
+    synchronized(systems) {
+      s.addDisconnectListener(
+        new InternalDistributedSystem.DisconnectListener() {
+          @Override
+          public String toString() {
+            return "Disconnect listener for InternalClientMembership";
+          }
+          
+          public void onDisconnect(InternalDistributedSystem ss) {
+            removeInternalDistributedSystem(ss);
+          }
+        });
+      systems.add(s);
+      // make sure executor is alive
+      ensureExecutorIsRunning(); // optimized to do nothing if already running
+    }
+  }
+
+  protected static void removeInternalDistributedSystem(InternalDistributedSystem sys) {
+    synchronized(systems) {
+      systems.remove(sys);
+      if (systems.isEmpty()) {
+        // clean up executor
+/*
+Object[] queueElementsBefore = new Object[executorQueue.size()];
+queueElementsBefore = executorQueue.toArray(queueElementsBefore);
+System.out.println("Before shut down, the executor's queue contains the following " + queueElementsBefore.length + " elements");
+for (int i=0; i<queueElementsBefore.length; i++) {
+  System.out.println("\t" + queueElementsBefore[i]);
+}
+*/
+        if (executor != null) {
+          executor.shutdown();
+        }
+/*
+Object[] queueElementsAfter = new Object[executorQueue.size()];
+queueElementsAfter = executorQueue.toArray(queueElementsAfter);
+System.out.println("After shut down, the executor's queue contains the following " + queueElementsAfter.length + " elements");
+for (int i=0; i<queueElementsAfter.length; i++) {
+  System.out.println("\t" + queueElementsAfter[i]);
+}
+*/
+        // deadcoded this clear to fix bug 35675 - clearing removed the shutdown token from the queue!
+        // executorQueue.clear();
+        executor = null;
+      }
+    }
+  }
+
+  /**
+   * @guarded.By {@link #systems}
+   */
+  private static void ensureExecutorIsRunning() {
+    // protected by calling method synchronized on systems
+    if (executor == null) {
+      final ThreadGroup group = threadGroup;
+      ThreadFactory tf = new ThreadFactory() {
+          public Thread newThread(Runnable command) {
+            Thread thread =
+                new Thread(group, command, "ClientMembership Event Invoker");
+            thread.setDaemon(true);
+            return thread;
+          }
+        };
+      LinkedBlockingQueue q = new LinkedBlockingQueue();
+      executor = new ThreadPoolExecutor(1, 1/*max unused*/,
+                                        15, TimeUnit.SECONDS, q, tf);
+    }
+  }
+
+  /**
+   * Internal implementation of ClientMembershipEvent.
+   */
+  protected static class InternalClientMembershipEvent
+  implements ClientMembershipEvent {
+
+    private final DistributedMember member;
+    private final boolean client;
+
+    /** Constructs new instance of event */
+    protected InternalClientMembershipEvent(DistributedMember member, boolean client) {
+      this.member = member;
+      this.client = client;
+    }
+
+    public DistributedMember getMember() {
+      return this.member;
+    }
+
+    public String getMemberId() {
+      return this.member == null ? "unknown" : this.member.getId();
+    }
+
+    public boolean isClient() {
+      return this.client;
+    }
+
+    @Override // GemStoneAddition
+    public String toString() {
+      final StringBuffer sb = new StringBuffer("[ClientMembershipEvent: ");
+      sb.append("member=").append(this.member);
+      sb.append(", isClient=").append(this.client);
+      sb.append("]");
+      return sb.toString();
+    }
+  }
+  
+  /** If set to true for testing then notification will be synchronous */
+  private static boolean forceSynchronous = false;
+  /** Set to true if synchronous notification is needed for testing */
+  public static void setForceSynchronous(boolean value) {
+    forceSynchronous = value;
+  }
+  
+  private static enum EventType{
+    CLIENT_JOINED,
+    CLIENT_LEFT,
+    CLIENT_CRASHED
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
index aac9176..74cdfa9 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
@@ -58,7 +58,7 @@ import com.gemstone.gemfire.ToDataException;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
 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.wan.GatewayTransportFilter;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -91,8 +91,6 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
  * Implements the acceptor thread on the bridge server. Accepts connections from
  * the edge and starts up threads to process requests from these.
  * 
- * @see com.gemstone.gemfire.cache.util.BridgeServer
- * 
  * @author Sudhir Menon
  * @since 2.0.2
  */
@@ -328,7 +326,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
     }
     {
       int tmp_maxThreads = maxThreads;
-      if (maxThreads == BridgeServer.DEFAULT_MAX_THREADS) {
+      if (maxThreads == CacheServer.DEFAULT_MAX_THREADS) {
         // consult system properties for 5.0.2 backwards compatibility
         if (DEPRECATED_SELECTOR) {
           tmp_maxThreads = DEPRECATED_SELECTOR_POOL_SIZE;
@@ -534,7 +532,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
       this.localPort = port;
       String sockName = this.serverSock.getLocalSocketAddress().toString();
       logger.info(LocalizedMessage.create(
-          LocalizedStrings.AcceptorImpl_BRIDGE_SERVER_CONNECTION_LISTENER_BOUND_TO_ADDRESS_0_WITH_BACKLOG_1,
+          LocalizedStrings.AcceptorImpl_CACHE_SERVER_CONNECTION_LISTENER_BOUND_TO_ADDRESS_0_WITH_BACKLOG_1,
           new Object[] {sockName, Integer.valueOf(backLog)}));
       if(isGatewayReceiver){
         this.stats = GatewayReceiverStats.createGatewayReceiverStats(sockName);
@@ -1357,7 +1355,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
         if (isRunning()) {
           if (!this.loggedAcceptError) {
             this.loggedAcceptError = true;
-            logger.error(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_BRIDGE_SERVER_UNEXPECTED_IOEXCEPTION_FROM_ACCEPT, e));
+            logger.error(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_CACHE_SERVER_UNEXPECTED_IOEXCEPTION_FROM_ACCEPT, e));
           }
           // Why sleep?
           // try {Thread.sleep(3000);} catch (InterruptedException ie) {}
@@ -1370,7 +1368,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
       catch (Exception e) {
         closeSocket(s);
         if (isRunning()) {
-          logger.fatal(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_BRIDGE_SERVER_UNEXPECTED_EXCEPTION, e));
+          logger.fatal(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_CACHE_SERVER_UNEXPECTED_EXCEPTION, e));
         }
       }
     }
@@ -1410,10 +1408,10 @@ public class AcceptorImpl extends Acceptor implements Runnable
                 if (!AcceptorImpl.this.loggedAcceptError) {
                   AcceptorImpl.this.loggedAcceptError = true;
                   if (ex instanceof SocketTimeoutException) {
-                    logger.warn(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_BRIDGE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION_DUE_TO_SOCKET_TIMEOUT));
+                    logger.warn(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_CACHE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION_DUE_TO_SOCKET_TIMEOUT));
                   }
                   else {
-                    logger.warn(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_BRIDGE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION__0, ex), ex);
+                    logger.warn(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_CACHE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION__0, ex), ex);
                   }
                 }
               }
@@ -1487,7 +1485,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
         SystemTimer.SystemTimerTask st = new SystemTimer.SystemTimerTask() {
           @Override
           public void run2() {
-            logger.warn(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_BRIDGE_SERVER_TIMED_OUT_WAITING_FOR_HANDSHAKE_FROM__0, s.getRemoteSocketAddress()));
+            logger.warn(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_CACHE_SERVER_TIMED_OUT_WAITING_FOR_HANDSHAKE_FROM__0, s.getRemoteSocketAddress()));
             closeSocket(s);
           }
         };
@@ -1636,7 +1634,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
     try {
       synchronized (syncLock) {
         this.shutdown = true;
-        logger.info(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_BRIDGE_SERVER_ON_PORT_0_IS_SHUTTING_DOWN, this.localPort)); 
+        logger.info(LocalizedMessage.create(LocalizedStrings.AcceptorImpl_CACHE_SERVER_ON_PORT_0_IS_SHUTTING_DOWN, this.localPort)); 
         if (this.thread != null) {
           this.thread.interrupt();
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
index deddfd1..b1464b9 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
@@ -77,10 +77,10 @@ import com.gemstone.gemfire.internal.SocketCloser;
 import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
-import com.gemstone.gemfire.internal.cache.BridgeRegionEventImpl;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientRegionEventImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.CacheClientStatus;
 import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor;
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
@@ -1103,7 +1103,7 @@ public class CacheClientNotifier {
           removeClientProxy(proxy);
 
           if (PoolImpl.AFTER_QUEUE_DESTROY_MESSAGE_FLAG) {
-            BridgeObserver bo = BridgeObserverHolder.getInstance();
+            ClientServerObserver bo = ClientServerObserverHolder.getInstance();
             bo.afterQueueDestroyMessage();
           }
 
@@ -1159,8 +1159,8 @@ public class CacheClientNotifier {
       RegionEventImpl regionEvent = (RegionEventImpl)event;
       callbackArgument = regionEvent.getRawCallbackArgument();
       eventIdentifier = regionEvent.getEventId();
-      if (event instanceof BridgeRegionEventImpl) {
-        BridgeRegionEventImpl bridgeEvent = (BridgeRegionEventImpl)event;
+      if (event instanceof ClientRegionEventImpl) {
+        ClientRegionEventImpl bridgeEvent = (ClientRegionEventImpl)event;
         membershipID = bridgeEvent.getContext();
       }
     }
@@ -2138,7 +2138,7 @@ public class CacheClientNotifier {
         && !HARegionQueue.HA_EVICTION_POLICY_NONE.equals(overflowAttributesList
             .get(0))) {
       haContainer = new HAContainerRegion(cache.getRegion(Region.SEPARATOR
-          + BridgeServerImpl.clientMessagesRegion((GemFireCacheImpl)cache,
+          + CacheServerImpl.clientMessagesRegion((GemFireCacheImpl)cache,
               (String)overflowAttributesList.get(0),
               ((Integer)overflowAttributesList.get(1)).intValue(),
               ((Integer)overflowAttributesList.get(2)).intValue(),

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
index 85c7493..9cfbbbe 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
@@ -66,8 +66,8 @@ import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.SystemTimer;
 import com.gemstone.gemfire.internal.SystemTimer.SystemTimerTask;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisee;
 import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor.InitialImageAdvice;
 import com.gemstone.gemfire.internal.cache.Conflatable;
@@ -2796,7 +2796,7 @@ public class CacheClientProxy implements ClientSession {
             latestValue);
         
         if (AFTER_MESSAGE_CREATION_FLAG) {
-          BridgeObserver bo = BridgeObserverHolder.getInstance();
+          ClientServerObserver bo = ClientServerObserverHolder.getInstance();
           bo.afterMessageCreation(message);
         }
      }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
index 64731d1..07dc030 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientUpdater.java
@@ -65,8 +65,8 @@ import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.StatisticsTypeFactoryImpl;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -1273,7 +1273,7 @@ public class CacheClientUpdater extends Thread implements ClientUpdater,
       
       // // CALLBACK TESTING PURPOSE ONLY ////
       if (PoolImpl.IS_INSTANTIATOR_CALLBACK) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.afterReceivingFromServer(eventId);
       }
       // /////////////////////////////////////
@@ -1323,7 +1323,7 @@ public class CacheClientUpdater extends Thread implements ClientUpdater,
       
       // // CALLBACK TESTING PURPOSE ONLY ////
       if (PoolImpl.IS_INSTANTIATOR_CALLBACK) {
-        BridgeObserver bo = BridgeObserverHolder.getInstance();
+        ClientServerObserver bo = ClientServerObserverHolder.getInstance();
         bo.afterReceivingFromServer(eventId);
       }
      ///////////////////////////////////////
@@ -1794,7 +1794,7 @@ public class CacheClientUpdater extends Thread implements ClientUpdater,
             if (errMessage == null) {
               errMessage = "";
             }
-            BridgeObserver bo = BridgeObserverHolder.getInstance();
+            ClientServerObserver bo = ClientServerObserverHolder.getInstance();
             bo.beforeFailoverByCacheClientUpdater(this.location);
             eManager.serverCrashed(this.endpoint);
             if (isDebugEnabled) {
@@ -1811,7 +1811,7 @@ public class CacheClientUpdater extends Thread implements ClientUpdater,
         catch (Exception e) {
           if (!quitting()) {
             this.endPointDied = true;
-            BridgeObserver bo = BridgeObserverHolder.getInstance();
+            ClientServerObserver bo = ClientServerObserverHolder.getInstance();
             bo.beforeFailoverByCacheClientUpdater(this.location);
             eManager.serverCrashed(this.endpoint);
             String message = ": Caught the following exception and will exit: ";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientBlacklistProcessor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientBlacklistProcessor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientBlacklistProcessor.java
index 01bcfac..73c3731 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientBlacklistProcessor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientBlacklistProcessor.java
@@ -17,7 +17,7 @@ import com.gemstone.gemfire.distributed.internal.PooledDistributionMessage;
 import com.gemstone.gemfire.distributed.internal.ReplyException;
 import com.gemstone.gemfire.distributed.internal.ReplyMessage;
 import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 
 import java.io.DataInput;
@@ -96,7 +96,7 @@ public class ClientBlacklistProcessor extends ReplyProcessor21 {
           if (l != null) {
             Iterator i = l.iterator();
             while (i.hasNext()) {
-              BridgeServerImpl bs = (BridgeServerImpl)i.next();
+              CacheServerImpl bs = (CacheServerImpl)i.next();
               CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier(); 
               //add client to the black list.
               ccn.addToBlacklistedClient(this.proxyId);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/RemoveClientFromBlacklistMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/RemoveClientFromBlacklistMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/RemoveClientFromBlacklistMessage.java
index 3e57a1b..6f0e033 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/RemoveClientFromBlacklistMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/RemoveClientFromBlacklistMessage.java
@@ -21,7 +21,7 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.PooledDistributionMessage;
-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.logging.LogService;
 /**
@@ -60,7 +60,7 @@ public class RemoveClientFromBlacklistMessage extends PooledDistributionMessage
       if (l != null) {
         Iterator i = l.iterator();
         while (i.hasNext()) {
-          BridgeServerImpl bs = (BridgeServerImpl)i.next();
+          CacheServerImpl bs = (CacheServerImpl)i.next();
           CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
           Set s = ccn.getBlacklistedClient();
           if (s != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
index b307b95..9fe595b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
@@ -47,7 +47,7 @@ import com.gemstone.gemfire.internal.cache.tier.Acceptor;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.ClientHandShake;
 import com.gemstone.gemfire.internal.cache.tier.Command;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.cache.tier.sockets.command.Default;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -571,7 +571,7 @@ public class ServerConnection implements Runnable {
       this.crHelper.checkCancelInProgress(null);
       if (clientJoined && isFiringMembershipEvents()) {
         // This is a new client. Notify bridge membership and heartbeat monitor.
-        InternalBridgeMembership.notifyJoined(this.proxyId.getDistributedMember(),
+        InternalClientMembership.notifyJoined(this.proxyId.getDistributedMember(),
             true);
         }
 
@@ -871,9 +871,9 @@ public class ServerConnection implements Runnable {
         // the heartbeat monitor; other wise just remove the connection.
         if (clientDeparted && isFiringMembershipEvents()) {
           if (this.clientDisconnectedCleanly && !forceClientCrashEvent) {
-            InternalBridgeMembership.notifyLeft(proxyId.getDistributedMember(), true);
+            InternalClientMembership.notifyLeft(proxyId.getDistributedMember(), true);
           } else {
-            InternalBridgeMembership.notifyCrashed(this.proxyId.getDistributedMember(), true);
+            InternalClientMembership.notifyCrashed(this.proxyId.getDistributedMember(), true);
           }
           // The client has departed. Remove this last connection and unregister it.
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/doc-files/communication-architecture.fig
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/doc-files/communication-architecture.fig b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/doc-files/communication-architecture.fig
index 0d169e9..b334ebb 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/doc-files/communication-architecture.fig
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/doc-files/communication-architecture.fig
@@ -119,10 +119,10 @@ Single
 4 0 0 50 -1 0 12 0.0000 4 180 1245 600 3900 ConnectionProxy\001
 -6
 6 600 2775 1725 3075
-# HREF="../../../../cache/util/BridgeLoader.html" ALT="BridgeLoader"
+# HREF="../../../../cache/client/Pool.html" ALT="ClientPool"
 2 2 0 1 0 7 50 -1 -1 0.000 0 0 -1 0 0 5
 	 600 2775 1725 2775 1725 3075 600 3075 600 2775
-4 0 0 50 -1 -1 12 0.0000 4 180 975 675 3000 BridgeLoader\001
+4 0 0 50 -1 -1 12 0.0000 4 180 975 675 3000 ClientPool\001
 -6
 2 1 2 1 0 7 50 -1 -1 3.000 0 0 -1 0 0 2
 	 6450 375 6450 6750


[32/38] incubator-geode git commit: GEODE-391, GEODE-398: fix race condition in TXExpiryJUnitTest

Posted by bs...@apache.org.
GEODE-391, GEODE-398: fix race condition in TXExpiryJUnitTest

The test now handled ExpiryTask reschedule.


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

Branch: refs/heads/feature/GEODE-77
Commit: 8d1ada30648acce2e09211ac20cfcb935ad77a89
Parents: 17fdf57
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Oct 13 16:12:55 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Oct 13 16:12:55 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/EntryExpiryTask.java |  3 ++
 .../gemfire/internal/cache/ExpiryTask.java      | 24 ++++++++-
 .../internal/cache/RegionExpiryTask.java        |  3 ++
 .../com/gemstone/gemfire/TXExpiryJUnitTest.java | 51 +++++++++++++++-----
 4 files changed, 68 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8d1ada30/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryExpiryTask.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryExpiryTask.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryExpiryTask.java
index ca9a4ef..ef47514 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryExpiryTask.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryExpiryTask.java
@@ -196,6 +196,9 @@ public class EntryExpiryTask extends ExpiryTask {
     }
     if (getExpirationTime() > 0) {
       addExpiryTask();
+      if (expiryTaskListener != null) {
+        expiryTaskListener.afterReschedule(this);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8d1ada30/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
index 357c0a8..9d5795a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
@@ -223,7 +223,11 @@ public abstract class ExpiryTask extends SystemTimer.SystemTimerTask {
   {
     ExpirationAction action = getAction();
     if (action == null) return false;
-    return expire(action, isPending);
+    boolean result = expire(action, isPending);
+    if (result && expiryTaskListener != null) {
+      expiryTaskListener.afterExpire(this);
+    }
+    return result;
   }
   
   /** Why did this expire?
@@ -380,7 +384,7 @@ public abstract class ExpiryTask extends SystemTimer.SystemTimerTask {
        logger.fatal(LocalizedMessage.create(LocalizedStrings.ExpiryTask_EXCEPTION_IN_EXPIRATION_TASK), ex);
     } finally {
       if (expiryTaskListener != null) {
-        expiryTaskListener.afterExpire(this);
+        expiryTaskListener.afterTaskRan(this);
       }
     }
   }
@@ -503,6 +507,22 @@ public abstract class ExpiryTask extends SystemTimer.SystemTimerTask {
    */
   public interface ExpiryTaskListener {
     /**
+     * Called after the given expiry task has run.
+     * This means that the time it was originally
+     * scheduled to run has elapsed and the scheduler
+     * has run the task. While running the task it
+     * may decide to expire it or reschedule it.
+     */
+    public void afterTaskRan(ExpiryTask et);
+    /**
+     * Called after the given expiry task has been
+     * rescheduled. afterTaskRan can still be called
+     * on the same task.
+     * In some cases a task is rescheduled without expiring it.
+     * In others it is expired and rescheduled.
+     */
+    public void afterReschedule(ExpiryTask et);
+    /**
      * Called after the given expiry task has expired.
      */
     public void afterExpire(ExpiryTask et);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8d1ada30/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionExpiryTask.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionExpiryTask.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionExpiryTask.java
index 68c4e3e..e17ad58 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionExpiryTask.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionExpiryTask.java
@@ -120,6 +120,9 @@ abstract class RegionExpiryTask extends ExpiryTask
     }
 
     addExpiryTask();
+    if (expiryTaskListener != null) {
+      expiryTaskListener.afterReschedule(this);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8d1ada30/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
index 867ebb2..ae94002 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
@@ -231,21 +231,27 @@ public class TXExpiryJUnitTest {
   }
   
   private void waitForEntryExpiration(LocalRegion lr, String key) {
-    ExpirationDetector detector = new ExpirationDetector(lr.getEntryExpiryTask(key));
-    ExpiryTask.expiryTaskListener = detector;
     try {
-      ExpiryTask.permitExpiration();
-      DistributedTestCase.waitForCriterion(detector, 3000, 2, true);
+      ExpirationDetector detector;
+      do {
+        detector = new ExpirationDetector(lr.getEntryExpiryTask(key));
+        ExpiryTask.expiryTaskListener = detector;
+        ExpiryTask.permitExpiration();
+        DistributedTestCase.waitForCriterion(detector, 3000, 2, true);
+      } while (!detector.hasExpired() && detector.wasRescheduled());
     } finally {
       ExpiryTask.expiryTaskListener = null;
     }
   }
   private void waitForRegionExpiration(LocalRegion lr, boolean ttl) {
-    ExpirationDetector detector = new ExpirationDetector(ttl ? lr.getRegionTTLExpiryTask() : lr.getRegionIdleExpiryTask());
-    ExpiryTask.expiryTaskListener = detector;
     try {
-      ExpiryTask.permitExpiration();
-      DistributedTestCase.waitForCriterion(detector, 3000, 2, true);
+      ExpirationDetector detector;
+      do {
+        detector = new ExpirationDetector(ttl ? lr.getRegionTTLExpiryTask() : lr.getRegionIdleExpiryTask());
+        ExpiryTask.expiryTaskListener = detector;
+        ExpiryTask.permitExpiration();
+        DistributedTestCase.waitForCriterion(detector, 3000, 2, true);
+      } while (!detector.hasExpired() && detector.wasRescheduled());
     } finally {
       ExpiryTask.expiryTaskListener = null;
     }
@@ -256,25 +262,48 @@ public class TXExpiryJUnitTest {
    * Used to detect that a particular ExpiryTask has expired.
    */
   public static class ExpirationDetector implements ExpiryTaskListener, WaitCriterion {
+    private volatile boolean ran = false;
     private volatile boolean expired = false;
-    private final ExpiryTask et;
+    private volatile boolean rescheduled = false;
+    public final ExpiryTask et;
     public ExpirationDetector(ExpiryTask et) {
       assertNotNull(et);
       this.et = et;
     }
     @Override
+    public void afterReschedule(ExpiryTask et) {
+      if (et == this.et) {
+        if (!hasExpired()) {
+          ExpiryTask.suspendExpiration();
+        }
+        this.rescheduled = true;
+      }
+    }
+    @Override
     public void afterExpire(ExpiryTask et) {
       if (et == this.et) {
         this.expired = true;
       }
     }
     @Override
+    public void afterTaskRan(ExpiryTask et) {
+      if (et == this.et) {
+        this.ran = true;
+      }
+    }
+    @Override
     public boolean done() {
-      return this.expired;
+      return this.ran;
     }
     @Override
     public String description() {
-      return "the expiry task " + this.et + " did not expire";
+      return "the expiry task " + this.et + " never ran";
+    }
+    public boolean wasRescheduled() {
+      return this.rescheduled;
+    }
+    public boolean hasExpired() {
+      return this.expired;
     }
   }
 


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java
index 6f2dfae..5ff95f0 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelTestBase.java
@@ -24,15 +24,14 @@ 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.client.internal.RegisterInterestTracker;
-import com.gemstone.gemfire.cache.util.BridgeServer;
-import com.gemstone.gemfire.cache.util.BridgeWriter;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
+import com.gemstone.gemfire.internal.cache.CacheServerImpl;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
@@ -77,7 +76,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
 
   static PoolImpl pool = null;  
 
-  static BridgeObserver oldBo = null;
+  static ClientServerObserver oldBo = null;
   
   static boolean FailOverDetectionByCCU = false;
   
@@ -151,12 +150,12 @@ public class RedundancyLevelTestBase extends DistributedTestCase
   public static void verifyDispatcherIsAlive()
   {
     try {
-//      assertEquals("More than one BridgeServer", 1, cache.getBridgeServers()
+//      assertEquals("More than one BridgeServer", 1, cache.getCacheServers()
 //          .size());
       WaitCriterion wc = new WaitCriterion() {
         String excuse;
         public boolean done() {
-          return cache.getBridgeServers().size() == 1;
+          return cache.getCacheServers().size() == 1;
         }
         public String description() {
           return excuse;
@@ -164,7 +163,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
       };
       DistributedTestCase.waitForCriterion(wc, 3 * 60 * 1000, 1000, true);
 
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
           .iterator().next();
       assertNotNull(bs);
       assertNotNull(bs.getAcceptor());
@@ -211,11 +210,11 @@ public class RedundancyLevelTestBase extends DistributedTestCase
   {
     try {
       // assertEquals("More than one BridgeServer", 1,
-      // cache.getBridgeServers().size());
+      // cache.getCacheServers().size());
       WaitCriterion wc = new WaitCriterion() {
         String excuse;
         public boolean done() {
-          return cache.getBridgeServers().size() == 1;
+          return cache.getCacheServers().size() == 1;
         }
         public String description() {
           return excuse;
@@ -223,7 +222,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
       };
       DistributedTestCase.waitForCriterion(wc, 3 * 60 * 1000, 1000, true);
 
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
           .iterator().next();
       assertNotNull(bs);
       assertNotNull(bs.getAcceptor());
@@ -351,9 +350,9 @@ public class RedundancyLevelTestBase extends DistributedTestCase
   
   public static void verifyNoCCP()
   {
-    assertEquals("More than one BridgeServer", 1, cache.getBridgeServers()
+    assertEquals("More than one BridgeServer", 1, cache.getCacheServers()
         .size());
-    BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
+    CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
         .iterator().next();
     assertNotNull(bs);
     assertNotNull(bs.getAcceptor());
@@ -369,7 +368,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
       WaitCriterion wc = new WaitCriterion() {
         String excuse;
         public boolean done() {
-          return cache.getBridgeServers().size() == 1;
+          return cache.getCacheServers().size() == 1;
         }
         public String description() {
           return excuse;
@@ -377,7 +376,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
       };
       DistributedTestCase.waitForCriterion(wc, 3 * 60 * 1000, 1000, true);
       
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
           .iterator().next();
       
       assertNotNull(bs);
@@ -406,16 +405,16 @@ public class RedundancyLevelTestBase extends DistributedTestCase
     try {
       WaitCriterion wc = new WaitCriterion() {
         public boolean done() {
-          return cache.getBridgeServers().size() == 1;
+          return cache.getCacheServers().size() == 1;
         }
         public String description() {
           return "Number of bridge servers (" 
-              + cache.getBridgeServers().size() + ") never became 1";
+              + cache.getCacheServers().size() + ") never became 1";
         }
       };
       DistributedTestCase.waitForCriterion(wc, 180 * 1000, 2000, true);
 
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
+      CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers()
           .iterator().next();
       assertNotNull(bs);
       assertNotNull(bs.getAcceptor());
@@ -475,9 +474,9 @@ public class RedundancyLevelTestBase extends DistributedTestCase
   public static void stopServer()
   {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         server.stop();
       }
     }
@@ -490,7 +489,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
   {
     try {
       Cache c = CacheFactory.getAnyInstance();
-      BridgeServerImpl bs = (BridgeServerImpl)c.getBridgeServers().iterator()
+      CacheServerImpl bs = (CacheServerImpl)c.getCacheServers().iterator()
           .next();
       assertNotNull(bs);
       bs.start();
@@ -526,7 +525,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
     
     if(!FailOverDetectionByCCU)
     {
-        oldBo = BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+        oldBo = ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
         public void beforeFailoverByCacheClientUpdater(ServerLocation epFailed)
         {
           try {
@@ -580,7 +579,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
     RegionAttributes attrs = factory.createRegionAttributes();
     cache.createVMRegion(REGION_NAME, attrs);
 
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server1.setMaximumTimeBetweenPings(180000);
@@ -618,7 +617,7 @@ public class RedundancyLevelTestBase extends DistributedTestCase
       super.tearDown2();
     
       if(!FailOverDetectionByCCU)
-        BridgeObserverHolder.setInstance(oldBo);   
+        ClientServerObserverHolder.setInstance(oldBo);   
     
       FailOverDetectionByCCU = false;
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegionCloseDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegionCloseDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegionCloseDUnitTest.java
index d6f950b..d4bfac7 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegionCloseDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegionCloseDUnitTest.java
@@ -16,12 +16,12 @@ 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.cache.server.CacheServer;
 
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 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.tier.sockets.CacheClientProxy;
 import com.gemstone.gemfire.cache.client.*;
 
@@ -125,7 +125,7 @@ public class RegionCloseDUnitTest extends DistributedTestCase
     factory.setDataPolicy(DataPolicy.REPLICATE);
     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);
@@ -137,10 +137,10 @@ public class RegionCloseDUnitTest extends DistributedTestCase
   public static void VerifyClientProxyOnServerBeforeClose()
   {
     Cache c = CacheFactory.getAnyInstance();
-    assertEquals("More than one BridgeServer", 1, c.getBridgeServers().size());
+    assertEquals("More than one CacheServer", 1, c.getCacheServers().size());
 
 
-    final BridgeServerImpl bs = (BridgeServerImpl) c.getBridgeServers().iterator().next();
+    final CacheServerImpl bs = (CacheServerImpl) c.getCacheServers().iterator().next();
     WaitCriterion ev = new WaitCriterion() {
       public boolean done() {
         return bs.getAcceptor().getCacheClientNotifier().getClientProxies().size() == 1;
@@ -182,7 +182,7 @@ public class RegionCloseDUnitTest extends DistributedTestCase
     final Cache c = CacheFactory.getAnyInstance();
     WaitCriterion ev = new WaitCriterion() {
       public boolean done() {
-        return c.getBridgeServers().size() == 1;
+        return c.getCacheServers().size() == 1;
       }
       public String description() {
         return null;
@@ -190,7 +190,7 @@ public class RegionCloseDUnitTest extends DistributedTestCase
     };
     DistributedTestCase.waitForCriterion(ev, 40 * 1000, 200, true);
 
-    final BridgeServerImpl bs = (BridgeServerImpl)c.getBridgeServers().iterator()
+    final CacheServerImpl bs = (CacheServerImpl)c.getCacheServers().iterator()
         .next();
     ev = new WaitCriterion() {
       public boolean done() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestBeforeRegionCreationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestBeforeRegionCreationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestBeforeRegionCreationDUnitTest.java
index 722f370..75fcfc6 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestBeforeRegionCreationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestBeforeRegionCreationDUnitTest.java
@@ -21,7 +21,7 @@ import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 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.CacheObserverAdapter;
 import com.gemstone.gemfire.internal.cache.CacheObserverHolder;
 import com.gemstone.gemfire.cache.client.*;
@@ -214,7 +214,7 @@ public class RegisterInterestBeforeRegionCreationDUnitTest extends DistributedTe
       RegionAttributes attrs = factory.createRegionAttributes();
       cache.createVMRegion(REGION_NAME, attrs);
     }
-    BridgeServerImpl server = (BridgeServerImpl)cache.addBridgeServer();
+    CacheServerImpl server = (CacheServerImpl)cache.addCacheServer();
     assertNotNull(server);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
@@ -267,4 +267,4 @@ public class RegisterInterestBeforeRegionCreationDUnitTest extends DistributedTe
       cache.getDistributedSystem().disconnect();
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestKeysDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestKeysDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestKeysDUnitTest.java
index 299e157..27e254d 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestKeysDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/RegisterInterestKeysDUnitTest.java
@@ -10,7 +10,7 @@ package com.gemstone.gemfire.internal.cache.tier.sockets;
 import java.util.*;
 
 import com.gemstone.gemfire.cache.*;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -171,7 +171,7 @@ public class RegisterInterestKeysDUnitTest extends DistributedTestCase
    
     RegionAttributes attrs = impl.createServerCacheAttributes();
     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/tier/sockets/ReliableMessagingDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
index 987182c..13f8d21 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/ReliableMessagingDUnitTest.java
@@ -21,13 +21,13 @@ 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.client.internal.QueueStateImpl.SequenceIdAndExpirationObject;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeObserver;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserver;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 import com.gemstone.gemfire.internal.cache.ha.HAHelper;
 import com.gemstone.gemfire.internal.cache.ha.HARegionQueue;
 import com.gemstone.gemfire.internal.cache.ha.ThreadIdentifier;
@@ -101,7 +101,7 @@ public class ReliableMessagingDUnitTest extends DistributedTestCase
   public void testPeriodicAckSendByClientPrimaryFailover() throws Exception {    
     addExpectedException("java.net.ConnectException");
     createEntries();
-    setBridgeObserverForBeforeSendingClientAck();    
+    setClientServerObserverForBeforeSendingClientAck();    
     server1.invoke(ReliableMessagingDUnitTest.class, "putOnServer");
     getLogWriter().info("Entering waitForServerUpdate");
     waitForServerUpdate();    
@@ -249,9 +249,9 @@ public class ReliableMessagingDUnitTest extends DistributedTestCase
   public static void stopServer()
   {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         server.stop();
       }
     }
@@ -279,10 +279,10 @@ public class ReliableMessagingDUnitTest extends DistributedTestCase
     }
   }
   
-  public static void setBridgeObserverForBeforeSendingClientAck() throws Exception
+  public static void setClientServerObserverForBeforeSendingClientAck() throws Exception
   {
     PoolImpl.BEFORE_SENDING_CLIENT_ACK_CALLBACK_FLAG = true;
-    origObserver = BridgeObserverHolder.setInstance(new BridgeObserverAdapter() {
+    origObserver = ClientServerObserverHolder.setInstance(new ClientServerObserverAdapter() {
       public void beforeSendingClientAck()
       {
         getLogWriter().info("beforeSendingClientAck invoked");
@@ -357,7 +357,7 @@ public class ReliableMessagingDUnitTest extends DistributedTestCase
     cache.setMessageSyncInterval(25);
     cache.createRegion(REGION_NAME, attrs);
 
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
     server.setNotifyBySubscription(true);
@@ -418,10 +418,10 @@ public class ReliableMessagingDUnitTest extends DistributedTestCase
     }
   }
   
-  private static BridgeObserver origObserver;
+  private static ClientServerObserver origObserver;
   
   public static void resetCallBack()  {    
-    BridgeObserverHolder.setInstance(origObserver);
+    ClientServerObserverHolder.setInstance(origObserver);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/TestPut.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/TestPut.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/TestPut.java
deleted file mode 100755
index a9e8861..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/TestPut.java
+++ /dev/null
@@ -1,53 +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.tier.sockets;
- 
-import java.io.IOException;
-
-import com.gemstone.gemfire.internal.cache.tier.Command;
-import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
-import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
-import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
-
-/**
- * @author Pallavi
- * 
- * TestPut is a dummy command to verify Command handling in BackwardCompatibilityComamndDUnitTest.
- */
-
- public class TestPut implements Command {
-
-   public TestPut() {	
-   }
-			  
-   final public void execute(Message msg, ServerConnection servConn) {
-	 // set flag true - to be checked in test
-	 BackwardCompatibilityCommandDUnitDisabledTest.TEST_PUT_COMMAND_INVOKED = true;
-	 
-	 // write reply to clients 
-	 servConn.setAsTrue(REQUIRES_RESPONSE);
-	 writeReply(msg, servConn);
-	 servConn.setAsTrue(RESPONDED);	 
-   }
-
-   private void writeReply(Message origMsg, ServerConnection servConn) {
-     Message replyMsg = servConn.getReplyMessage();
-     replyMsg.setMessageType(MessageType.REPLY);
-     replyMsg.setNumberOfParts(1);
-     replyMsg.setTransactionId(origMsg.getTransactionId());
-     replyMsg.addBytesPart(BaseCommand.OK_BYTES);
-     try {
-       replyMsg.send();
-     } 
-     catch (IOException ioe){
-       ioe.printStackTrace();
-     }
-   }
- }
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
index 1eb3ae3..30c0f1b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/UpdatePropagationDUnitTest.java
@@ -28,7 +28,7 @@ import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.Connection;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -402,9 +402,9 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase
   public static void killServer(Integer port )
   {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         if(server.getPort() == port.intValue()){
           server.stop();
         }
@@ -419,7 +419,7 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase
   public static void startServer(Integer port)
   {
     try {
-      BridgeServer server1 = cache.addBridgeServer();
+      CacheServer server1 = cache.addCacheServer();
       server1.setPort(port.intValue());
       server1.setNotifyBySubscription(true);
       server1.start();
@@ -498,7 +498,7 @@ public class UpdatePropagationDUnitTest extends DistributedTestCase
     new UpdatePropagationDUnitTest("temp").createCache(new Properties());
     RegionAttributes attrs = impl.createCacheServerAttributes(); 
     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/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java
index 9eca096..a850b21 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/VerifyUpdatesFromNonInterestEndPointDUnitTest.java
@@ -17,7 +17,7 @@ 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.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -191,7 +191,7 @@ public class VerifyUpdatesFromNonInterestEndPointDUnitTest extends DistributedTe
     factory.setDataPolicy(DataPolicy.REPLICATE);
     RegionAttributes attrs = factory.create();
     cache.createRegion(REGION_NAME, attrs);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET) ;
     server1.setPort(port);
     server1.setNotifyBySubscription(true);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java
index 54867d1..c90e68f 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/UniversalMembershipListenerAdapterDUnitTest.java
@@ -23,14 +23,14 @@ import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.ServerConnectivityException;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache30.BridgeTestCase;
+import com.gemstone.gemfire.cache30.ClientServerTestCase;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DurableClientAttributes;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
@@ -52,7 +52,7 @@ import dunit.VM;
  * @since 4.2.1
  */
 
-public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase {
+public class UniversalMembershipListenerAdapterDUnitTest extends ClientServerTestCase {
   protected static final boolean CLIENT = true;
   protected static final boolean SERVER = false;
   
@@ -81,7 +81,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
   @Override
   public void tearDown2() throws Exception {
     super.tearDown2();
-    InternalBridgeMembership.unregisterAllListeners();
+    InternalClientMembership.unregisterAllListeners();
   }
   
   /**
@@ -115,7 +115,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
     };
     
     DistributedMember clientJoined = new TestDistributedMember("clientJoined");
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
+    InternalClientMembership.notifyJoined(clientJoined, true);
     synchronized(listener) {
       if (!fired[0]) {
         listener.wait(SYNC_ASYNC_EVENT_WAIT_MILLIS);
@@ -173,7 +173,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
     DistributedMember memberA = new TestDistributedMember("memberA");
     
     // first join
-    InternalBridgeMembership.notifyJoined(memberA, true);
+    InternalClientMembership.notifyJoined(memberA, true);
     synchronized(listener) {
       if (!fired[JOINED]) {
         listener.wait(SYNC_ASYNC_EVENT_WAIT_MILLIS);
@@ -187,14 +187,14 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
     memberId[JOINED] = null;
 
     // duplicate join
-    InternalBridgeMembership.notifyJoined(memberA, true);
+    InternalClientMembership.notifyJoined(memberA, true);
     pause(BRIEF_PAUSE_MILLIS);
     assertFalse(fired[JOINED]);
     assertNull(member[JOINED]);
     assertNull(memberId[JOINED]);
 
     // first left
-    InternalBridgeMembership.notifyLeft(memberA, true);
+    InternalClientMembership.notifyLeft(memberA, true);
     synchronized(listener) {
       if (!fired[LEFT]) {
         listener.wait(SYNC_ASYNC_EVENT_WAIT_MILLIS);
@@ -208,14 +208,14 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
     memberId[LEFT] = null;
 
     // duplicate left
-    InternalBridgeMembership.notifyLeft(memberA, true);
+    InternalClientMembership.notifyLeft(memberA, true);
     pause(BRIEF_PAUSE_MILLIS);
     assertFalse(fired[LEFT]);
     assertNull(member[LEFT]);
     assertNull(memberId[LEFT]);
     
     // rejoin
-    InternalBridgeMembership.notifyJoined(memberA, true);
+    InternalClientMembership.notifyJoined(memberA, true);
     synchronized(listener) {
       if (!fired[JOINED]) {
         listener.wait(SYNC_ASYNC_EVENT_WAIT_MILLIS);
@@ -410,7 +410,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
         getSystem(config);
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, getServerHostName(host), ports, false, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, false, -1, -1, null);
         createRegion(name, factory.create());
         assertNotNull(getRootRegion().getSubregion(name));
       }
@@ -870,7 +870,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
         assertFalse(getCache().isClosed());
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, getServerHostName(host), ports, false, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, false, -1, -1, null);
         createRegion(name, factory.create());
         assertNotNull(getRootRegion().getSubregion(name));
       }
@@ -1327,7 +1327,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
    * com.gemstone.gemfire.internal.cache.tier.Endpoint#getNumConnections
    * Endpoint.getNumConnections()} to {@link 
    * com.gemstone.gemfire.internal.cache.tier.Endpoint}. Note: This probably
-   * won't work if the BridgeLoader has more than one Endpoint.
+   * won't work if the pool has more than one Endpoint.
    */
   protected void waitForClientToFullyConnect(final PoolImpl pool) {
     getLogWriter().info("[waitForClientToFullyConnect]");
@@ -1864,7 +1864,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
     
     vm0.invoke(createBridgeServer);
     
-    // gather details for later creation of BridgeLoader...
+    // gather details for later creation of pool...
     assertEquals(ports[0],
                  vm0.invokeInt(UniversalMembershipListenerAdapterDUnitTest.class, 
                                "getTestServerEventsInLonerClient_port"));
@@ -1880,7 +1880,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
     // create region which connects to bridge server
     AttributesFactory factory = new AttributesFactory();
     factory.setScope(Scope.LOCAL);
-    BridgeTestCase.configureConnectionPool(factory, getServerHostName(host), ports, false, -1, -1, null);
+    ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, false, -1, -1, null);
     createRegion(name, factory.create());
     assertNotNull(getRootRegion().getSubregion(name));
 
@@ -2035,7 +2035,7 @@ public class UniversalMembershipListenerAdapterDUnitTest extends BridgeTestCase
     // reconnect bridge client to test for crashed event
     vm0.invoke(createBridgeServer);
     
-    // gather details for later creation of BridgeLoader...
+    // gather details for later creation of pool...
     assertEquals(ports[0],
                  vm0.invokeInt(UniversalMembershipListenerAdapterDUnitTest.class, 
                                "getTestServerEventsInLonerClient_port"));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/PdxFormatterPutGetJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/PdxFormatterPutGetJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/PdxFormatterPutGetJUnitTest.java
index 8a9bd0f..f70d2d2 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/PdxFormatterPutGetJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/PdxFormatterPutGetJUnitTest.java
@@ -27,7 +27,7 @@ import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
@@ -47,7 +47,7 @@ public class PdxFormatterPutGetJUnitTest {
     this.c = (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").setPdxReadSerialized(true).create();
     
     //start cache-server
-    BridgeServer server = c.addBridgeServer();
+    CacheServer server = c.addCacheServer();
     final int serverPort = 40405;
     server.setPort(serverPort);
     server.start();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/dunit/DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/DistributedTestCase.java b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
index c78510a..d9f88ca 100755
--- a/gemfire-core/src/test/java/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
@@ -63,7 +63,7 @@ import com.gemstone.gemfire.internal.cache.HARegion;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.tier.sockets.DataSerializerPropogationDUnitTest;
@@ -799,7 +799,7 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
     LogWrapper.close();
     ClientProxyMembershipID.system = null;
     MultiVMRegionTestCase.CCRegion = null;
-    InternalBridgeMembership.unregisterAllListeners();
+    InternalClientMembership.unregisterAllListeners();
     ClientStatsManager.cleanupForTests();
     unregisterInstantiatorsInThisVM();
     GemFireTracer.DEBUG = Boolean.getBoolean("DistributionManager.DEBUG_JAVAGROUPS");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt
index f24eff5..060a5ec 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/excludedClasses.txt
@@ -91,7 +91,7 @@ com/gemstone/gemfire/internal/shared/OSType$4
 com/gemstone/gemfire/internal/shared/OSType$5
 com/gemstone/gemfire/internal/shared/TCPSocketOptions
 com/gemstone/gemfire/internal/process/SharedConfigurationNotAvailableException
-com/gemstone/gemfire/internal/cache/tier/InternalBridgeMembership$EventType
+com/gemstone/gemfire/internal/cache/tier/InternalClientMembership$EventType
 com/gemstone/gemfire/internal/util/concurrent/StoppableReadWriteLock
 com/gemstone/gemfire/rest/internal/web/exception.GemfireRestException
 com/gemstone/gemfire/rest/internal/web/controllers/support/JSONTypes

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index 79f5e22..490117f 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -34,7 +34,7 @@ com/gemstone/gemfire/cache/SubscriptionAttributes,2
 fromData,14,2a2bb9000f0100b80010b50003b1
 toData,14,2b2ab40003b4000db9000e0200b1
 
-com/gemstone/gemfire/cache/client/internal/BridgeServerLoadMessage,2
+com/gemstone/gemfire/cache/client/internal/CacheServerLoadMessage,2
 fromData,52,2a2bb7000c2abb000d59b7000eb500022ab400022bb8000f2abb001059b70011b500032ab400032bb8000f2a2bb80012b50004b1
 toData,30,2a2bb700132ab400022bb800142ab400032bb800142ab400042bb80015b1
 
@@ -844,11 +844,11 @@ com/gemstone/gemfire/internal/cache/AddCacheServerProfileMessage,2
 fromData,16,2a2bb7002f2a2bb900300100b50007b1
 toData,16,2a2bb7002d2b2ab40007b9002e0200b1
 
-com/gemstone/gemfire/internal/cache/BridgeRegionEventImpl,2
+com/gemstone/gemfire/internal/cache/ClientRegionEventImpl,2
 fromData,14,2a2bb700132a2bb80014b60003b1
 toData,14,2a2bb700112ab6000d2bb80012b1
 
-com/gemstone/gemfire/internal/cache/BridgeServerAdvisor$BridgeServerProfile,2
+com/gemstone/gemfire/internal/cache/CacheServerAdvisor$CacheServerProfile,2
 fromData,53,2a2bb700102a2bb80011b500042a2bb900120100b500062abb001359b70014b500052ab400052bb800152a2bb900160100b60017b1
 toData,42,2a2bb7000a2ab400042bb8000b2b2ab40006b9000c02002ab400052bb8000d2b2ab6000eb9000f0300b1
 
@@ -1164,8 +1164,8 @@ fromData,207,2a2bb9004f0100b500162a2bb900500100b80051b500032a2bb80052b500182a2bb
 toData,178,2b2ab40016b9004602002b2ab40003b40047b9004802002ab400182bb800492b2ab40005b9004a02002b2ab4001fb9004a02002ab400042bb800492ab400082bb8004b2ab400092bb8004b2ab400072bb8004b2ab4000c2bb800492ab4000e2bb800492ab4000d2bb800492ab400102bb800492ab4000f2bb800492b2ab40006b9004a02002ab400112bb8004c2ab400142bb8004d2ab40001b9004e010099000b012bb8004ca7000b2ab400012bb8004cb1
 
 com/gemstone/gemfire/internal/cache/PoolFactoryImpl$PoolAttributes,2
-fromData,153,2a2bb80033b500022a2bb80033b500042a2bb80033b500062a2bb80033b500082a2bb80033b500092a2bb80033b5000a2a2bb80033b5000e2a2bb80034b5000d2a2bb80034b500112a2bb80033b500152a2bb80033b500172a2bb80035b500072a2bb80035b500132a2bb80036b5001a2a2bb80037b5001e2a2bb80037b5001f2a2bb80035b500232a2bb80033b500122a2bb80035b5001bb1
-toData,153,2ab400022bb8002e2ab400042bb8002e2ab400062bb8002e2ab400082bb8002e2ab400092bb8002e2ab4000a2bb8002e2ab4000e2bb8002e2ab4000d2bb8002f2ab400112bb8002f2ab400152bb8002e2ab400172bb8002e2ab400072bb800302ab400132bb800302ab4001a2bb800312ab4001e2bb800322ab4001f2bb800322ab400232bb800302ab400122bb8002e2ab4001b2bb80030b1
+fromData,145,2a2bb80031b500022a2bb80031b500042a2bb80031b500062a2bb80031b500082a2bb80031b500092a2bb80031b5000a2a2bb80031b5000e2a2bb80032b5000d2a2bb80032b500112a2bb80031b500152a2bb80031b500172a2bb80033b500072a2bb80033b500132a2bb80034b5001a2a2bb80035b5001e2a2bb80035b5001f2a2bb80031b500122a2bb80033b5001bb1
+toData,145,2ab400022bb8002c2ab400042bb8002c2ab400062bb8002c2ab400082bb8002c2ab400092bb8002c2ab4000a2bb8002c2ab4000e2bb8002c2ab4000d2bb8002d2ab400112bb8002d2ab400152bb8002c2ab400172bb8002c2ab400072bb8002e2ab400132bb8002e2ab4001a2bb8002f2ab4001e2bb800302ab4001f2bb800302ab400122bb8002c2ab4001b2bb8002eb1
 
 com/gemstone/gemfire/internal/cache/PreferBytesCachedDeserializable,2
 fromData,9,2a2bb8000cb50002b1

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
index f272e4a..58ef0ab 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
@@ -218,15 +218,6 @@ com/gemstone/gemfire/cache/query/internal/utils/PDXUtils$1,false
 com/gemstone/gemfire/cache/query/internal/utils/PDXUtils$2,false
 com/gemstone/gemfire/cache/snapshot/SnapshotOptions$SnapshotFormat,false
 com/gemstone/gemfire/cache/util/BoundedLinkedHashMap,true,-3419897166186852692,_maximumNumberOfEntries:int
-com/gemstone/gemfire/cache/util/BridgeWriterException,true,-295001316745954159
-com/gemstone/gemfire/cache/util/EndpointDoesNotExistException,true,1654241470788247283
-com/gemstone/gemfire/cache/util/EndpointException,false
-com/gemstone/gemfire/cache/util/EndpointExistsException,true,950617116786308012
-com/gemstone/gemfire/cache/util/EndpointInUseException,true,-4087729485272321469
-com/gemstone/gemfire/cache/util/IncompatibleVersionException,true,7008667865037538081
-com/gemstone/gemfire/cache/util/ServerRefusedConnectionException,true,-4996327025772566931
-com/gemstone/gemfire/cache/util/UnknownVersionException,true,7379530185697556990
-com/gemstone/gemfire/cache/util/VersionException,false
 com/gemstone/gemfire/cache/wan/GatewaySender$OrderPolicy,false
 com/gemstone/gemfire/cache/util/Gateway$OrderPolicy,false
 com/gemstone/gemfire/compression/CompressionException,true,4118639654597191235

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml
index 4e884d5..38ad448 100644
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/internal/cache/tier/sockets/RedundancyLevelJUnitTest.xml
@@ -6,25 +6,16 @@
 -->
 
 <!DOCTYPE cache PUBLIC
-  "-//GemStone Systems, Inc.//GemFire Declarative Caching 5.0//EN"
-  "http://www.gemstone.com/dtd/cache5_0.dtd">
+  "-//GemStone Systems, Inc.//GemFire Declarative Caching 8.0//EN"
+  "http://www.gemstone.com/dtd/cache8_0.dtd">
 <cache>
+  <pool name="clientPool" subscription-enabled="true" subscription-redundancy="6">
+    <server host="localhost" port="30303"/>
+  </pool>
   <region name="root">
     <region-attributes/> 
     <region name="exampleRegion">
-      <region-attributes scope="distributed-ack">
-        <cache-writer>
-          <class-name>com.gemstone.gemfire.cache.util.BridgeWriter</class-name>
-          <parameter name="endpoints">
-            <string>server=localhost:30303</string>
-          </parameter>
-          <parameter name="establishCallbackConnection">
-            <string>true</string>
-          </parameter>
-          <parameter name="redundancyLevel">
-            <string>6</string>
-          </parameter>          
-        </cache-writer>
+      <region-attributes scope="distributed-ack" pool-name="clientPool">
       </region-attributes>
     </region>
   </region>



[03/38] incubator-geode git commit: GEODE-378: use SynchronousQueue to prevent deadlock

Posted by bs...@apache.org.
GEODE-378: use SynchronousQueue to prevent deadlock

The SynchronousQueueNoSpin class has been completely removed.
The JDK SynchronousQueue is now used in its place.


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

Branch: refs/heads/feature/GEODE-77
Commit: 9e4a94e06377094336cb31882b96e8aff2ebe8dd
Parents: 35d3f06
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Oct 1 11:45:37 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Fri Oct 2 09:36:45 2015 -0700

----------------------------------------------------------------------
 .../FunctionExecutionPooledExecutor.java        |    7 +-
 .../internal/PooledExecutorWithDMStats.java     |    7 +-
 .../util/concurrent/SynchronousQueueNoSpin.java | 1144 ------------------
 3 files changed, 6 insertions(+), 1152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9e4a94e0/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/FunctionExecutionPooledExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/FunctionExecutionPooledExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/FunctionExecutionPooledExecutor.java
index 501b83c..698bc86 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/FunctionExecutionPooledExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/FunctionExecutionPooledExecutor.java
@@ -21,7 +21,6 @@ import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.java.util.concurrent.SynchronousQueueNoSpin;
 
 import java.util.List;
 
@@ -75,10 +74,10 @@ public class FunctionExecutionPooledExecutor extends ThreadPoolExecutor {
   Thread bufferConsumer;
   
   private static BlockingQueue<Runnable> initQ(BlockingQueue<Runnable> q) {
-    if (q instanceof SynchronousQueue || q instanceof SynchronousQueueNoSpin) {
+    if (q instanceof SynchronousQueue) {
       return q;
     } else {
-      return new SynchronousQueueNoSpin<Runnable>();
+      return new SynchronousQueue<Runnable>();
     }
   }
 
@@ -124,7 +123,7 @@ public class FunctionExecutionPooledExecutor extends ThreadPoolExecutor {
     }
     else {
 
-      if (q instanceof SynchronousQueue || q instanceof SynchronousQueueNoSpin) {
+      if (q instanceof SynchronousQueue) {
         return new CallerRunsPolicy();
         // return new BlockHandler();
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9e4a94e0/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java
index 05f9dd5..a50e594 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/PooledExecutorWithDMStats.java
@@ -17,7 +17,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.ThreadFactory;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.java.util.concurrent.SynchronousQueueNoSpin;
 
 import java.util.List;
 
@@ -55,15 +54,15 @@ public class PooledExecutorWithDMStats extends ThreadPoolExecutor {
   private Thread bufferConsumer;
   
   private static BlockingQueue<Runnable> initQ(BlockingQueue<Runnable> q) {
-    if (q instanceof SynchronousQueue || q instanceof SynchronousQueueNoSpin) {
+    if (q instanceof SynchronousQueue) {
       return q;
     } else {
-      return new SynchronousQueueNoSpin<Runnable>();
+      return new SynchronousQueue<Runnable>();
     }
   }
 
   private static RejectedExecutionHandler initREH(BlockingQueue<Runnable> q) {
-    if (q instanceof SynchronousQueue || q instanceof SynchronousQueueNoSpin) {
+    if (q instanceof SynchronousQueue) {
       return new CallerRunsPolicy();
       //return new BlockHandler();
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9e4a94e0/gemfire-core/src/main/java/com/gemstone/java/util/concurrent/SynchronousQueueNoSpin.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/java/util/concurrent/SynchronousQueueNoSpin.java b/gemfire-core/src/main/java/com/gemstone/java/util/concurrent/SynchronousQueueNoSpin.java
deleted file mode 100644
index ca2c1d9..0000000
--- a/gemfire-core/src/main/java/com/gemstone/java/util/concurrent/SynchronousQueueNoSpin.java
+++ /dev/null
@@ -1,1144 +0,0 @@
-/*
- * Written by Doug Lea, Bill Scherer, and Michael Scott with
- * assistance from members of JCP JSR-166 Expert Group and released to
- * the public domain, as explained at
- * http://creativecommons.org/licenses/publicdomain
- * 
- */
-
-package com.gemstone.java.util.concurrent;
-import java.util.AbstractQueue;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-import java.util.concurrent.locks.LockSupport;
-import java.util.concurrent.locks.ReentrantLock;
-
-/**
- * A {@linkplain BlockingQueue blocking queue} in which each insert
- * operation must wait for a corresponding remove operation by another
- * thread, and vice versa.  A synchronous queue does not have any
- * internal capacity, not even a capacity of one.  You cannot
- * <tt>peek</tt> at a synchronous queue because an element is only
- * present when you try to remove it; you cannot insert an element
- * (using any method) unless another thread is trying to remove it;
- * you cannot iterate as there is nothing to iterate.  The
- * <em>head</em> of the queue is the element that the first queued
- * inserting thread is trying to add to the queue; if there is no such
- * queued thread then no element is available for removal and
- * <tt>poll()</tt> will return <tt>null</tt>.  For purposes of other
- * <tt>Collection</tt> methods (for example <tt>contains</tt>), a
- * <tt>SynchronousQueue</tt> acts as an empty collection.  This queue
- * does not permit <tt>null</tt> elements.
- *
- * <p>Synchronous queues are similar to rendezvous channels used in
- * CSP and Ada. They are well suited for handoff designs, in which an
- * object running in one thread must sync up with an object running
- * in another thread in order to hand it some information, event, or
- * task.
- *
- * <p> This class supports an optional fairness policy for ordering
- * waiting producer and consumer threads.  By default, this ordering
- * is not guaranteed. However, a queue constructed with fairness set
- * to <tt>true</tt> grants threads access in FIFO order.
- *
- * <p>This class and its iterator implement all of the
- * <em>optional</em> methods of the {@link Collection} and {@link
- * Iterator} interfaces.
- *
- * <p>This class is a member of the
- * <a href="{@docRoot}/../technotes/guides/collections/index.html">
- * Java Collections Framework</a>.
- *
- * @since 1.5
- * @author Doug Lea and Bill Scherer and Michael Scott
- * @param <E> the type of elements held in this collection
- * 
- * GemstoneAddition
- * ------------------
- * This version was taken from the jsr166 CVS repository
- * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/src/main/java/util/concurrent/SynchronousQueue.java revision 1.68
- * 
- * The JSR version is identical java 1.6_10 version except for formatting
- * and licensing.
- * 
- * It has been modified by gemstone to remove the spin waiting which is hard coded
- * into the java 1.6 version. We found through performance testing
- * that this spin waiting was causing up to a 50% throughput decrease
- * for some of our perf tests (eg perf081).
- * 
- * In addition, the calls to LockSupport.park(Object) have been changed
- * to LockSupport.park() to compile with java 1.5. 
- */
-public class SynchronousQueueNoSpin<E> extends AbstractQueue<E>
-    implements BlockingQueue<E>, java.io.Serializable {
-    private static final long serialVersionUID = -3223113410248163686L;
-
-    /*
-     * This class implements extensions of the dual stack and dual
-     * queue algorithms described in "Nonblocking Concurrent Objects
-     * with Condition Synchronization", by W. N. Scherer III and
-     * M. L. Scott.  18th Annual Conf. on Distributed Computing,
-     * Oct. 2004 (see also
-     * http://www.cs.rochester.edu/u/scott/synchronization/pseudocode/duals.html).
-     * The (Lifo) stack is used for non-fair mode, and the (Fifo)
-     * queue for fair mode. The performance of the two is generally
-     * similar. Fifo usually supports higher throughput under
-     * contention but Lifo maintains higher thread locality in common
-     * applications.
-     *
-     * A dual queue (and similarly stack) is one that at any given
-     * time either holds "data" -- items provided by put operations,
-     * or "requests" -- slots representing take operations, or is
-     * empty. A call to "fulfill" (i.e., a call requesting an item
-     * from a queue holding data or vice versa) dequeues a
-     * complementary node.  The most interesting feature of these
-     * queues is that any operation can figure out which mode the
-     * queue is in, and act accordingly without needing locks.
-     *
-     * Both the queue and stack extend abstract class Transferer
-     * defining the single method transfer that does a put or a
-     * take. These are unified into a single method because in dual
-     * data structures, the put and take operations are symmetrical,
-     * so nearly all code can be combined. The resulting transfer
-     * methods are on the long side, but are easier to follow than
-     * they would be if broken up into nearly-duplicated parts.
-     *
-     * The queue and stack data structures share many conceptual
-     * similarities but very few concrete details. For simplicity,
-     * they are kept distinct so that they can later evolve
-     * separately.
-     *
-     * The algorithms here differ from the versions in the above paper
-     * in extending them for use in synchronous queues, as well as
-     * dealing with cancellation. The main differences include:
-     *
-     *  1. The original algorithms used bit-marked pointers, but
-     *     the ones here use mode bits in nodes, leading to a number
-     *     of further adaptations.
-     *  2. SynchronousQueues must block threads waiting to become
-     *     fulfilled.
-     *  3. Support for cancellation via timeout and interrupts,
-     *     including cleaning out cancelled nodes/threads
-     *     from lists to avoid garbage retention and memory depletion.
-     *
-     * Blocking is mainly accomplished using LockSupport park/unpark,
-     * except that nodes that appear to be the next ones to become
-     * fulfilled first spin a bit (on multiprocessors only). On very
-     * busy synchronous queues, spinning can dramatically improve
-     * throughput. And on less busy ones, the amount of spinning is
-     * small enough not to be noticeable.
-     *
-     * Cleaning is done in different ways in queues vs stacks.  For
-     * queues, we can almost always remove a node immediately in O(1)
-     * time (modulo retries for consistency checks) when it is
-     * cancelled. But if it may be pinned as the current tail, it must
-     * wait until some subsequent cancellation. For stacks, we need a
-     * potentially O(n) traversal to be sure that we can remove the
-     * node, but this can run concurrently with other threads
-     * accessing the stack.
-     *
-     * While garbage collection takes care of most node reclamation
-     * issues that otherwise complicate nonblocking algorithms, care
-     * is taken to "forget" references to data, other nodes, and
-     * threads that might be held on to long-term by blocked
-     * threads. In cases where setting to null would otherwise
-     * conflict with main algorithms, this is done by changing a
-     * node's link to now point to the node itself. This doesn't arise
-     * much for Stack nodes (because blocked threads do not hang on to
-     * old head pointers), but references in Queue nodes must be
-     * aggressively forgotten to avoid reachability of everything any
-     * node has ever referred to since arrival.
-     */
-
-    /**
-     * Shared internal API for dual stacks and queues.
-     */
-    static abstract class Transferer {
-        /**
-         * Performs a put or take.
-         *
-         * @param e if non-null, the item to be handed to a consumer;
-         *          if null, requests that transfer return an item
-         *          offered by producer.
-         * @param timed if this operation should timeout
-         * @param nanos the timeout, in nanoseconds
-         * @return if non-null, the item provided or received; if null,
-         *         the operation failed due to timeout or interrupt --
-         *         the caller can distinguish which of these occurred
-         *         by checking Thread.interrupted.
-         */
-        abstract Object transfer(Object e, boolean timed, long nanos);
-    }
-
-    /** The number of CPUs, for spin control */
-    static final int NCPUS = Runtime.getRuntime().availableProcessors();
-
-    /**
-     * The number of times to spin before blocking in timed waits.
-     * The value is empirically derived -- it works well across a
-     * variety of processors and OSes. Empirically, the best value
-     * seems not to vary with number of CPUs (beyond 2) so is just
-     * a constant.
-     */
-    //@GemstoneAddition
-    //static final int maxTimedSpins = (NCPUS < 2)? 0 : 32;
-    static final int maxTimedSpins = 0;
-
-    /**
-     * The number of times to spin before blocking in untimed waits.
-     * This is greater than timed value because untimed waits spin
-     * faster since they don't need to check times on each spin.
-     */
-    static final int maxUntimedSpins = maxTimedSpins * 16;
-
-    /**
-     * The number of nanoseconds for which it is faster to spin
-     * rather than to use timed park. A rough estimate suffices.
-     */
-    //@GemstoneAddition
-    //static final long spinForTimeoutThreshold = 1000L;
-    static final long spinForTimeoutThreshold = 0L;
-
-    /** Dual stack */
-    static final class TransferStack extends Transferer {
-        /*
-         * This extends Scherer-Scott dual stack algorithm, differing,
-         * among other ways, by using "covering" nodes rather than
-         * bit-marked pointers: Fulfilling operations push on marker
-         * nodes (with FULFILLING bit set in mode) to reserve a spot
-         * to match a waiting node.
-         */
-
-        /* Modes for SNodes, ORed together in node fields */
-        /** Node represents an unfulfilled consumer */
-        static final int REQUEST    = 0;
-        /** Node represents an unfulfilled producer */
-        static final int DATA       = 1;
-        /** Node is fulfilling another unfulfilled DATA or REQUEST */
-        static final int FULFILLING = 2;
-
-        /** Return true if m has fulfilling bit set */
-        static boolean isFulfilling(int m) { return (m & FULFILLING) != 0; }
-
-        /** Node class for TransferStacks. */
-        static final class SNode {
-            volatile SNode next;        // next node in stack
-            volatile SNode match;       // the node matched to this
-            volatile Thread waiter;     // to control park/unpark
-            Object item;                // data; or null for REQUESTs
-            int mode;
-            // Note: item and mode fields don't need to be volatile
-            // since they are always written before, and read after,
-            // other volatile/atomic operations.
-
-            SNode(Object item) {
-                this.item = item;
-            }
-
-            static final AtomicReferenceFieldUpdater<SNode, SNode>
-                nextUpdater = AtomicReferenceFieldUpdater.newUpdater
-                (SNode.class, SNode.class, "next");
-
-            boolean casNext(SNode cmp, SNode val) {
-                return (cmp == next &&
-                        nextUpdater.compareAndSet(this, cmp, val));
-            }
-
-            static final AtomicReferenceFieldUpdater<SNode, SNode>
-                matchUpdater = AtomicReferenceFieldUpdater.newUpdater
-                (SNode.class, SNode.class, "match");
-
-            /**
-             * Tries to match node s to this node, if so, waking up thread.
-             * Fulfillers call tryMatch to identify their waiters.
-             * Waiters block until they have been matched.
-             *
-             * @param s the node to match
-             * @return true if successfully matched to s
-             */
-            boolean tryMatch(SNode s) {
-                if (match == null &&
-                    matchUpdater.compareAndSet(this, null, s)) {
-                    Thread w = waiter;
-                    if (w != null) {    // waiters need at most one unpark
-                        waiter = null;
-                        LockSupport.unpark(w);
-                    }
-                    return true;
-                }
-                return match == s;
-            }
-
-            /**
-             * Tries to cancel a wait by matching node to itself.
-             */
-            void tryCancel() {
-                matchUpdater.compareAndSet(this, null, this);
-            }
-
-            boolean isCancelled() {
-                return match == this;
-            }
-        }
-
-        /** The head (top) of the stack */
-        volatile SNode head;
-
-        static final AtomicReferenceFieldUpdater<TransferStack, SNode>
-            headUpdater = AtomicReferenceFieldUpdater.newUpdater
-            (TransferStack.class,  SNode.class, "head");
-
-        boolean casHead(SNode h, SNode nh) {
-            return h == head && headUpdater.compareAndSet(this, h, nh);
-        }
-
-        /**
-         * Creates or resets fields of a node. Called only from transfer
-         * where the node to push on stack is lazily created and
-         * reused when possible to help reduce intervals between reads
-         * and CASes of head and to avoid surges of garbage when CASes
-         * to push nodes fail due to contention.
-         */
-        static SNode snode(SNode s, Object e, SNode next, int mode) {
-            if (s == null) s = new SNode(e);
-            s.mode = mode;
-            s.next = next;
-            return s;
-        }
-
-        /**
-         * Puts or takes an item.
-         */
-        Object transfer(Object e, boolean timed, long nanos) {
-            /*
-             * Basic algorithm is to loop trying one of three actions:
-             *
-             * 1. If apparently empty or already containing nodes of same
-             *    mode, try to push node on stack and wait for a match,
-             *    returning it, or null if cancelled.
-             *
-             * 2. If apparently containing node of complementary mode,
-             *    try to push a fulfilling node on to stack, match
-             *    with corresponding waiting node, pop both from
-             *    stack, and return matched item. The matching or
-             *    unlinking might not actually be necessary because of
-             *    other threads performing action 3:
-             *
-             * 3. If top of stack already holds another fulfilling node,
-             *    help it out by doing its match and/or pop
-             *    operations, and then continue. The code for helping
-             *    is essentially the same as for fulfilling, except
-             *    that it doesn't return the item.
-             */
-
-            SNode s = null; // constructed/reused as needed
-            int mode = (e == null)? REQUEST : DATA;
-
-            for (;;) {
-                SNode h = head;
-                if (h == null || h.mode == mode) {  // empty or same-mode
-                    if (timed && nanos <= 0) {      // can't wait
-                        if (h != null && h.isCancelled())
-                            casHead(h, h.next);     // pop cancelled node
-                        else
-                            return null;
-                    } else if (casHead(h, s = snode(s, e, h, mode))) {
-                        SNode m = awaitFulfill(s, timed, nanos);
-                        if (m == s) {               // wait was cancelled
-                            clean(s);
-                            return null;
-                        }
-                        if ((h = head) != null && h.next == s)
-                            casHead(h, s.next);     // help s's fulfiller
-                        return mode == REQUEST? m.item : s.item;
-                    }
-                } else if (!isFulfilling(h.mode)) { // try to fulfill
-                    if (h.isCancelled())            // already cancelled
-                        casHead(h, h.next);         // pop and retry
-                    else if (casHead(h, s=snode(s, e, h, FULFILLING|mode))) {
-                        for (;;) { // loop until matched or waiters disappear
-                            SNode m = s.next;       // m is s's match
-                            if (m == null) {        // all waiters are gone
-                                casHead(s, null);   // pop fulfill node
-                                s = null;           // use new node next time
-                                break;              // restart main loop
-                            }
-                            SNode mn = m.next;
-                            if (m.tryMatch(s)) {
-                                casHead(s, mn);     // pop both s and m
-                                return (mode == REQUEST)? m.item : s.item;
-                            } else                  // lost match
-                                s.casNext(m, mn);   // help unlink
-                        }
-                    }
-                } else {                            // help a fulfiller
-                    SNode m = h.next;               // m is h's match
-                    if (m == null)                  // waiter is gone
-                        casHead(h, null);           // pop fulfilling node
-                    else {
-                        SNode mn = m.next;
-                        if (m.tryMatch(h))          // help match
-                            casHead(h, mn);         // pop both h and m
-                        else                        // lost match
-                            h.casNext(m, mn);       // help unlink
-                    }
-                }
-            }
-        }
-
-        /**
-         * Spins/blocks until node s is matched by a fulfill operation.
-         *
-         * @param s the waiting node
-         * @param timed true if timed wait
-         * @param nanos timeout value
-         * @return matched node, or s if cancelled
-         */
-        SNode awaitFulfill(SNode s, boolean timed, long nanos) {
-            /*
-             * When a node/thread is about to block, it sets its waiter
-             * field and then rechecks state at least one more time
-             * before actually parking, thus covering race vs
-             * fulfiller noticing that waiter is non-null so should be
-             * woken.
-             *
-             * When invoked by nodes that appear at the point of call
-             * to be at the head of the stack, calls to park are
-             * preceded by spins to avoid blocking when producers and
-             * consumers are arriving very close in time.  This can
-             * happen enough to bother only on multiprocessors.
-             *
-             * The order of checks for returning out of main loop
-             * reflects fact that interrupts have precedence over
-             * normal returns, which have precedence over
-             * timeouts. (So, on timeout, one last check for match is
-             * done before giving up.) Except that calls from untimed
-             * SynchronousQueue.{poll/offer} don't check interrupts
-             * and don't wait at all, so are trapped in transfer
-             * method rather than calling awaitFulfill.
-             */
-            long lastTime = (timed)? System.nanoTime() : 0;
-            Thread w = Thread.currentThread();
-            SNode h = head;
-            int spins = (shouldSpin(s)?
-                         (timed? maxTimedSpins : maxUntimedSpins) : 0);
-            for (;;) {
-                if (w.isInterrupted())
-                    s.tryCancel();
-                SNode m = s.match;
-                if (m != null)
-                    return m;
-                if (timed) {
-                    long now = System.nanoTime();
-                    nanos -= now - lastTime;
-                    lastTime = now;
-                    if (nanos <= 0) {
-                        s.tryCancel();
-                        continue;
-                    }
-                }
-                if (spins > 0)
-                    spins = shouldSpin(s)? (spins-1) : 0;
-                else if (s.waiter == null)
-                    s.waiter = w; // establish waiter so can park next iter
-                else if (!timed)
-                    LockSupport.park();
-                else if (nanos > spinForTimeoutThreshold)
-                    LockSupport.parkNanos(nanos);
-            }
-        }
-
-        /**
-         * Returns true if node s is at head or there is an active
-         * fulfiller.
-         */
-        boolean shouldSpin(SNode s) {
-            SNode h = head;
-            return (h == s || h == null || isFulfilling(h.mode));
-        }
-
-        /**
-         * Unlinks s from the stack.
-         */
-        void clean(SNode s) {
-            s.item = null;   // forget item
-            s.waiter = null; // forget thread
-
-            /*
-             * At worst we may need to traverse entire stack to unlink
-             * s. If there are multiple concurrent calls to clean, we
-             * might not see s if another thread has already removed
-             * it. But we can stop when we see any node known to
-             * follow s. We use s.next unless it too is cancelled, in
-             * which case we try the node one past. We don't check any
-             * further because we don't want to doubly traverse just to
-             * find sentinel.
-             */
-
-            SNode past = s.next;
-            if (past != null && past.isCancelled())
-                past = past.next;
-
-            // Absorb cancelled nodes at head
-            SNode p;
-            while ((p = head) != null && p != past && p.isCancelled())
-                casHead(p, p.next);
-
-            // Unsplice embedded nodes
-            while (p != null && p != past) {
-                SNode n = p.next;
-                if (n != null && n.isCancelled())
-                    p.casNext(n, n.next);
-                else
-                    p = n;
-            }
-        }
-    }
-
-    /** Dual Queue */
-    static final class TransferQueue extends Transferer {
-        /*
-         * This extends Scherer-Scott dual queue algorithm, differing,
-         * among other ways, by using modes within nodes rather than
-         * marked pointers. The algorithm is a little simpler than
-         * that for stacks because fulfillers do not need explicit
-         * nodes, and matching is done by CAS'ing QNode.item field
-         * from non-null to null (for put) or vice versa (for take).
-         */
-
-        /** Node class for TransferQueue. */
-        static final class QNode {
-            volatile QNode next;          // next node in queue
-            volatile Object item;         // CAS'ed to or from null
-            volatile Thread waiter;       // to control park/unpark
-            final boolean isData;
-
-            QNode(Object item, boolean isData) {
-                this.item = item;
-                this.isData = isData;
-            }
-
-            static final AtomicReferenceFieldUpdater<QNode, QNode>
-                nextUpdater = AtomicReferenceFieldUpdater.newUpdater
-                (QNode.class, QNode.class, "next");
-
-            boolean casNext(QNode cmp, QNode val) {
-                return (next == cmp &&
-                        nextUpdater.compareAndSet(this, cmp, val));
-            }
-
-            static final AtomicReferenceFieldUpdater<QNode, Object>
-                itemUpdater = AtomicReferenceFieldUpdater.newUpdater
-                (QNode.class, Object.class, "item");
-
-            boolean casItem(Object cmp, Object val) {
-                return (item == cmp &&
-                        itemUpdater.compareAndSet(this, cmp, val));
-            }
-
-            /**
-             * Tries to cancel by CAS'ing ref to this as item.
-             */
-            void tryCancel(Object cmp) {
-                itemUpdater.compareAndSet(this, cmp, this);
-            }
-
-            boolean isCancelled() {
-                return item == this;
-            }
-
-            /**
-             * Returns true if this node is known to be off the queue
-             * because its next pointer has been forgotten due to
-             * an advanceHead operation.
-             */
-            boolean isOffList() {
-                return next == this;
-            }
-        }
-
-        /** Head of queue */
-        transient volatile QNode head;
-        /** Tail of queue */
-        transient volatile QNode tail;
-        /**
-         * Reference to a cancelled node that might not yet have been
-         * unlinked from queue because it was the last inserted node
-         * when it cancelled.
-         */
-        transient volatile QNode cleanMe;
-
-        TransferQueue() {
-            QNode h = new QNode(null, false); // initialize to dummy node.
-            head = h;
-            tail = h;
-        }
-
-        static final AtomicReferenceFieldUpdater<TransferQueue, QNode>
-            headUpdater = AtomicReferenceFieldUpdater.newUpdater
-            (TransferQueue.class,  QNode.class, "head");
-
-        /**
-         * Tries to cas nh as new head; if successful, unlink
-         * old head's next node to avoid garbage retention.
-         */
-        void advanceHead(QNode h, QNode nh) {
-            if (h == head && headUpdater.compareAndSet(this, h, nh))
-                h.next = h; // forget old next
-        }
-
-        static final AtomicReferenceFieldUpdater<TransferQueue, QNode>
-            tailUpdater = AtomicReferenceFieldUpdater.newUpdater
-            (TransferQueue.class, QNode.class, "tail");
-
-        /**
-         * Tries to cas nt as new tail.
-         */
-        void advanceTail(QNode t, QNode nt) {
-            if (tail == t)
-                tailUpdater.compareAndSet(this, t, nt);
-        }
-
-        static final AtomicReferenceFieldUpdater<TransferQueue, QNode>
-            cleanMeUpdater = AtomicReferenceFieldUpdater.newUpdater
-            (TransferQueue.class, QNode.class, "cleanMe");
-
-        /**
-         * Tries to CAS cleanMe slot.
-         */
-        boolean casCleanMe(QNode cmp, QNode val) {
-            return (cleanMe == cmp &&
-                    cleanMeUpdater.compareAndSet(this, cmp, val));
-        }
-
-        /**
-         * Puts or takes an item.
-         */
-        Object transfer(Object e, boolean timed, long nanos) {
-            /* Basic algorithm is to loop trying to take either of
-             * two actions:
-             *
-             * 1. If queue apparently empty or holding same-mode nodes,
-             *    try to add node to queue of waiters, wait to be
-             *    fulfilled (or cancelled) and return matching item.
-             *
-             * 2. If queue apparently contains waiting items, and this
-             *    call is of complementary mode, try to fulfill by CAS'ing
-             *    item field of waiting node and dequeuing it, and then
-             *    returning matching item.
-             *
-             * In each case, along the way, check for and try to help
-             * advance head and tail on behalf of other stalled/slow
-             * threads.
-             *
-             * The loop starts off with a null check guarding against
-             * seeing uninitialized head or tail values. This never
-             * happens in current SynchronousQueue, but could if
-             * callers held non-volatile/final ref to the
-             * transferer. The check is here anyway because it places
-             * null checks at top of loop, which is usually faster
-             * than having them implicitly interspersed.
-             */
-
-            QNode s = null; // constructed/reused as needed
-            boolean isData = (e != null);
-
-            for (;;) {
-                QNode t = tail;
-                QNode h = head;
-                if (t == null || h == null)         // saw uninitialized value
-                    continue;                       // spin
-
-                if (h == t || t.isData == isData) { // empty or same-mode
-                    QNode tn = t.next;
-                    if (t != tail)                  // inconsistent read
-                        continue;
-                    if (tn != null) {               // lagging tail
-                        advanceTail(t, tn);
-                        continue;
-                    }
-                    if (timed && nanos <= 0)        // can't wait
-                        return null;
-                    if (s == null)
-                        s = new QNode(e, isData);
-                    if (!t.casNext(null, s))        // failed to link in
-                        continue;
-
-                    advanceTail(t, s);              // swing tail and wait
-                    Object x = awaitFulfill(s, e, timed, nanos);
-                    if (x == s) {                   // wait was cancelled
-                        clean(t, s);
-                        return null;
-                    }
-
-                    if (!s.isOffList()) {           // not already unlinked
-                        advanceHead(t, s);          // unlink if head
-                        if (x != null)              // and forget fields
-                            s.item = s;
-                        s.waiter = null;
-                    }
-                    return (x != null)? x : e;
-
-                } else {                            // complementary-mode
-                    QNode m = h.next;               // node to fulfill
-                    if (t != tail || m == null || h != head)
-                        continue;                   // inconsistent read
-
-                    Object x = m.item;
-                    if (isData == (x != null) ||    // m already fulfilled
-                        x == m ||                   // m cancelled
-                        !m.casItem(x, e)) {         // lost CAS
-                        advanceHead(h, m);          // dequeue and retry
-                        continue;
-                    }
-
-                    advanceHead(h, m);              // successfully fulfilled
-                    LockSupport.unpark(m.waiter);
-                    return (x != null)? x : e;
-                }
-            }
-        }
-
-        /**
-         * Spins/blocks until node s is fulfilled.
-         *
-         * @param s the waiting node
-         * @param e the comparison value for checking match
-         * @param timed true if timed wait
-         * @param nanos timeout value
-         * @return matched item, or s if cancelled
-         */
-        Object awaitFulfill(QNode s, Object e, boolean timed, long nanos) {
-            /* Same idea as TransferStack.awaitFulfill */
-            long lastTime = (timed)? System.nanoTime() : 0;
-            Thread w = Thread.currentThread();
-            int spins = ((head.next == s) ?
-                         (timed? maxTimedSpins : maxUntimedSpins) : 0);
-            for (;;) {
-                if (w.isInterrupted())
-                    s.tryCancel(e);
-                Object x = s.item;
-                if (x != e)
-                    return x;
-                if (timed) {
-                    long now = System.nanoTime();
-                    nanos -= now - lastTime;
-                    lastTime = now;
-                    if (nanos <= 0) {
-                        s.tryCancel(e);
-                        continue;
-                    }
-                }
-                if (spins > 0)
-                    --spins;
-                else if (s.waiter == null)
-                    s.waiter = w;
-                else if (!timed)
-                    LockSupport.park();
-                else if (nanos > spinForTimeoutThreshold)
-                    LockSupport.parkNanos(nanos);
-            }
-        }
-
-        /**
-         * Gets rid of cancelled node s with original predecessor pred.
-         */
-        void clean(QNode pred, QNode s) {
-            s.waiter = null; // forget thread
-            /*
-             * At any given time, exactly one node on list cannot be
-             * deleted -- the last inserted node. To accommodate this,
-             * if we cannot delete s, we save its predecessor as
-             * "cleanMe", deleting the previously saved version
-             * first. At least one of node s or the node previously
-             * saved can always be deleted, so this always terminates.
-             */
-            while (pred.next == s) { // Return early if already unlinked
-                QNode h = head;
-                QNode hn = h.next;   // Absorb cancelled first node as head
-                if (hn != null && hn.isCancelled()) {
-                    advanceHead(h, hn);
-                    continue;
-                }
-                QNode t = tail;      // Ensure consistent read for tail
-                if (t == h)
-                    return;
-                QNode tn = t.next;
-                if (t != tail)
-                    continue;
-                if (tn != null) {
-                    advanceTail(t, tn);
-                    continue;
-                }
-                if (s != t) {        // If not tail, try to unsplice
-                    QNode sn = s.next;
-                    if (sn == s || pred.casNext(s, sn))
-                        return;
-                }
-                QNode dp = cleanMe;
-                if (dp != null) {    // Try unlinking previous cancelled node
-                    QNode d = dp.next;
-                    QNode dn;
-                    if (d == null ||               // d is gone or
-                        d == dp ||                 // d is off list or
-                        !d.isCancelled() ||        // d not cancelled or
-                        (d != t &&                 // d not tail and
-                         (dn = d.next) != null &&  //   has successor
-                         dn != d &&                //   that is on list
-                         dp.casNext(d, dn)))       // d unspliced
-                        casCleanMe(dp, null);
-                    if (dp == pred)
-                        return;      // s is already saved node
-                } else if (casCleanMe(null, pred))
-                    return;          // Postpone cleaning s
-            }
-        }
-    }
-
-    /**
-     * The transferer. Set only in constructor, but cannot be declared
-     * as final without further complicating serialization.  Since
-     * this is accessed only at most once per public method, there
-     * isn't a noticeable performance penalty for using volatile
-     * instead of final here.
-     */
-    private transient volatile Transferer transferer;
-
-    /**
-     * Creates a <tt>SynchronousQueue</tt> with nonfair access policy.
-     */
-    public SynchronousQueueNoSpin() {
-        this(false);
-    }
-
-    /**
-     * Creates a <tt>SynchronousQueue</tt> with the specified fairness policy.
-     *
-     * @param fair if true, waiting threads contend in FIFO order for
-     *        access; otherwise the order is unspecified.
-     */
-    public SynchronousQueueNoSpin(boolean fair) {
-        transferer = (fair)? new TransferQueue() : new TransferStack();
-    }
-
-    /**
-     * Adds the specified element to this queue, waiting if necessary for
-     * another thread to receive it.
-     *
-     * @throws InterruptedException {@inheritDoc}
-     * @throws NullPointerException {@inheritDoc}
-     */
-    public void put(E o) throws InterruptedException {
-        if (o == null) throw new NullPointerException();
-        if (transferer.transfer(o, false, 0) == null) {
-            Thread.interrupted();
-            throw new InterruptedException();
-        }
-    }
-
-    /**
-     * Inserts the specified element into this queue, waiting if necessary
-     * up to the specified wait time for another thread to receive it.
-     *
-     * @return <tt>true</tt> if successful, or <tt>false</tt> if the
-     *         specified waiting time elapses before a consumer appears.
-     * @throws InterruptedException {@inheritDoc}
-     * @throws NullPointerException {@inheritDoc}
-     */
-    public boolean offer(E o, long timeout, TimeUnit unit)
-        throws InterruptedException {
-        if (o == null) throw new NullPointerException();
-        if (transferer.transfer(o, true, unit.toNanos(timeout)) != null)
-            return true;
-        if (!Thread.interrupted())
-            return false;
-        throw new InterruptedException();
-    }
-
-    /**
-     * Inserts the specified element into this queue, if another thread is
-     * waiting to receive it.
-     *
-     * @param e the element to add
-     * @return <tt>true</tt> if the element was added to this queue, else
-     *         <tt>false</tt>
-     * @throws NullPointerException if the specified element is null
-     */
-    public boolean offer(E e) {
-        if (e == null) throw new NullPointerException();
-        return transferer.transfer(e, true, 0) != null;
-    }
-
-    /**
-     * Retrieves and removes the head of this queue, waiting if necessary
-     * for another thread to insert it.
-     *
-     * @return the head of this queue
-     * @throws InterruptedException {@inheritDoc}
-     */
-    public E take() throws InterruptedException {
-        Object e = transferer.transfer(null, false, 0);
-        if (e != null)
-            return (E)e;
-        Thread.interrupted();
-        throw new InterruptedException();
-    }
-
-    /**
-     * Retrieves and removes the head of this queue, waiting
-     * if necessary up to the specified wait time, for another thread
-     * to insert it.
-     *
-     * @return the head of this queue, or <tt>null</tt> if the
-     *         specified waiting time elapses before an element is present.
-     * @throws InterruptedException {@inheritDoc}
-     */
-    public E poll(long timeout, TimeUnit unit) throws InterruptedException {
-        Object e = transferer.transfer(null, true, unit.toNanos(timeout));
-        if (e != null || !Thread.interrupted())
-            return (E)e;
-        throw new InterruptedException();
-    }
-
-    /**
-     * Retrieves and removes the head of this queue, if another thread
-     * is currently making an element available.
-     *
-     * @return the head of this queue, or <tt>null</tt> if no
-     *         element is available.
-     */
-    public E poll() {
-        return (E)transferer.transfer(null, true, 0);
-    }
-
-    /**
-     * Always returns <tt>true</tt>.
-     * A <tt>SynchronousQueue</tt> has no internal capacity.
-     *
-     * @return <tt>true</tt>
-     */
-    public boolean isEmpty() {
-        return true;
-    }
-
-    /**
-     * Always returns zero.
-     * A <tt>SynchronousQueue</tt> has no internal capacity.
-     *
-     * @return zero.
-     */
-    public int size() {
-        return 0;
-    }
-
-    /**
-     * Always returns zero.
-     * A <tt>SynchronousQueue</tt> has no internal capacity.
-     *
-     * @return zero.
-     */
-    public int remainingCapacity() {
-        return 0;
-    }
-
-    /**
-     * Does nothing.
-     * A <tt>SynchronousQueue</tt> has no internal capacity.
-     */
-    public void clear() {
-    }
-
-    /**
-     * Always returns <tt>false</tt>.
-     * A <tt>SynchronousQueue</tt> has no internal capacity.
-     *
-     * @param o the element
-     * @return <tt>false</tt>
-     */
-    public boolean contains(Object o) {
-        return false;
-    }
-
-    /**
-     * Always returns <tt>false</tt>.
-     * A <tt>SynchronousQueue</tt> has no internal capacity.
-     *
-     * @param o the element to remove
-     * @return <tt>false</tt>
-     */
-    public boolean remove(Object o) {
-        return false;
-    }
-
-    /**
-     * Returns <tt>false</tt> unless the given collection is empty.
-     * A <tt>SynchronousQueue</tt> has no internal capacity.
-     *
-     * @param c the collection
-     * @return <tt>false</tt> unless given collection is empty
-     */
-    public boolean containsAll(Collection<?> c) {
-        return c.isEmpty();
-    }
-
-    /**
-     * Always returns <tt>false</tt>.
-     * A <tt>SynchronousQueue</tt> has no internal capacity.
-     *
-     * @param c the collection
-     * @return <tt>false</tt>
-     */
-    public boolean removeAll(Collection<?> c) {
-        return false;
-    }
-
-    /**
-     * Always returns <tt>false</tt>.
-     * A <tt>SynchronousQueue</tt> has no internal capacity.
-     *
-     * @param c the collection
-     * @return <tt>false</tt>
-     */
-    public boolean retainAll(Collection<?> c) {
-        return false;
-    }
-
-    /**
-     * Always returns <tt>null</tt>.
-     * A <tt>SynchronousQueue</tt> does not return elements
-     * unless actively waited on.
-     *
-     * @return <tt>null</tt>
-     */
-    public E peek() {
-        return null;
-    }
-
-    /**
-     * Returns an empty iterator in which <tt>hasNext</tt> always returns
-     * <tt>false</tt>.
-     *
-     * @return an empty iterator
-     */
-    public Iterator<E> iterator() {
-        return Collections.<E>emptyList().iterator();
-    }
-
-    /**
-     * Returns a zero-length array.
-     * @return a zero-length array
-     */
-    public Object[] toArray() {
-        return new Object[0];
-    }
-
-    /**
-     * Sets the zeroeth element of the specified array to <tt>null</tt>
-     * (if the array has non-zero length) and returns it.
-     *
-     * @param a the array
-     * @return the specified array
-     * @throws NullPointerException if the specified array is null
-     */
-    public <T> T[] toArray(T[] a) {
-        if (a.length > 0)
-            a[0] = null;
-        return a;
-    }
-
-    /**
-     * @throws UnsupportedOperationException {@inheritDoc}
-     * @throws ClassCastException            {@inheritDoc}
-     * @throws NullPointerException          {@inheritDoc}
-     * @throws IllegalArgumentException      {@inheritDoc}
-     */
-    public int drainTo(Collection<? super E> c) {
-        if (c == null)
-            throw new NullPointerException();
-        if (c == this)
-            throw new IllegalArgumentException();
-        int n = 0;
-        E e;
-        while ( (e = poll()) != null) {
-            c.add(e);
-            ++n;
-        }
-        return n;
-    }
-
-    /**
-     * @throws UnsupportedOperationException {@inheritDoc}
-     * @throws ClassCastException            {@inheritDoc}
-     * @throws NullPointerException          {@inheritDoc}
-     * @throws IllegalArgumentException      {@inheritDoc}
-     */
-    public int drainTo(Collection<? super E> c, int maxElements) {
-        if (c == null)
-            throw new NullPointerException();
-        if (c == this)
-            throw new IllegalArgumentException();
-        int n = 0;
-        E e;
-        while (n < maxElements && (e = poll()) != null) {
-            c.add(e);
-            ++n;
-        }
-        return n;
-    }
-
-    /*
-     * To cope with serialization strategy in the 1.5 version of
-     * SynchronousQueue, we declare some unused classes and fields
-     * that exist solely to enable serializability across versions.
-     * These fields are never used, so are initialized only if this
-     * object is ever serialized or deserialized.
-     */
-
-    static class WaitQueue implements java.io.Serializable {
-        private static final long serialVersionUID = 423369940180943459L;
-    }
-    static class LifoWaitQueue extends WaitQueue {
-        private static final long serialVersionUID = -3633113410248163686L;
-    }
-    static class FifoWaitQueue extends WaitQueue {
-        private static final long serialVersionUID = -3623113410248163686L;
-    }
-    private ReentrantLock qlock;
-    private WaitQueue waitingProducers;
-    private WaitQueue waitingConsumers;
-
-    /**
-     * Save the state to a stream (that is, serialize it).
-     *
-     * @param s the stream
-     */
-    private void writeObject(java.io.ObjectOutputStream s)
-        throws java.io.IOException {
-        boolean fair = transferer instanceof TransferQueue;
-        if (fair) {
-            qlock = new ReentrantLock(true);
-            waitingProducers = new FifoWaitQueue();
-            waitingConsumers = new FifoWaitQueue();
-        }
-        else {
-            qlock = new ReentrantLock();
-            waitingProducers = new LifoWaitQueue();
-            waitingConsumers = new LifoWaitQueue();
-        }
-        s.defaultWriteObject();
-    }
-
-    private void readObject(final java.io.ObjectInputStream s)
-        throws java.io.IOException, ClassNotFoundException {
-        s.defaultReadObject();
-        if (waitingProducers instanceof FifoWaitQueue)
-            transferer = new TransferQueue();
-        else
-            transferer = new TransferStack();
-    }
-
-}


[05/38] incubator-geode git commit: [GEODE-280] GroupByPartitionedQueryDUnitTest.testCompactRangeIndex fails intermittently Test was expecting sorted results but was not specifying group by in query clause Refactored test code Added additional debugging

Posted by bs...@apache.org.
[GEODE-280] GroupByPartitionedQueryDUnitTest.testCompactRangeIndex fails intermittently
Test was expecting sorted results but was not specifying group by in query clause
Refactored test code
Added additional debugging on failure
Removed excessive logging during test runs


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

Branch: refs/heads/feature/GEODE-77
Commit: fc9ee589b05709c820271ada0d336d3794becdb4
Parents: 4eed17b
Author: Jason Huynh <jh...@pivotal.io>
Authored: Mon Oct 5 09:52:54 2015 -0700
Committer: Jason Huynh <jh...@pivotal.io>
Committed: Mon Oct 5 09:52:54 2015 -0700

----------------------------------------------------------------------
 .../cache/query/functional/GroupByTestImpl.java |   2 +-
 .../query/functional/StructSetOrResultsSet.java | 223 +++++++------------
 2 files changed, 76 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fc9ee589/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/GroupByTestImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/GroupByTestImpl.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/GroupByTestImpl.java
index 47c31e2..51a3491 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/GroupByTestImpl.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/GroupByTestImpl.java
@@ -852,7 +852,7 @@ public abstract class GroupByTestImpl implements GroupByTestInterface {
     }
     QueryService qs = CacheUtils.getQueryService();
     String[] queries = {
-        "select pos.secId from  /portfolio  p, p.positions.values pos where NOT (pos.secId IN SET('SUN', 'ORCL')) ",// 6
+        "select pos.secId from  /portfolio  p, p.positions.values pos where NOT (pos.secId IN SET('SUN', 'ORCL')) group by pos.secId ",// 6
         "select pos.secId , count(pos.ID) from /portfolio p, p.positions.values pos where  pos.secId > 'APPL' group by pos.secId ",// 7
 
     };

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fc9ee589/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java
index 3593db0..d53c28b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/functional/StructSetOrResultsSet.java
@@ -64,29 +64,9 @@ public class StructSetOrResultsSet extends TestCase {
 
     Collection coll1 = null;
     Collection coll2 = null;
-    Iterator itert1 = null;
-    Iterator itert2 = null;
-    ObjectType type1, type2;
     for (int j = 0; j < len; j++) {
-      type1 = ((SelectResults) r[j][0]).getCollectionType().getElementType();
-      type2 = ((SelectResults) r[j][1]).getCollectionType().getElementType();
-      if ((type1.getClass().getName()).equals(type2.getClass().getName())) {
-        CacheUtils.log("Both SelectResults are of the same Type i.e.--> "
-            + ((SelectResults) r[j][0]).getCollectionType().getElementType());
-      } else {
-        CacheUtils.log("Classes are : " + type1.getClass().getName() + " "
-            + type2.getClass().getName());
-        fail("FAILED:Select result Type is different in both the cases."
-            + "; failed query=" + queries[j]);
-      }
-      if (((SelectResults) r[j][0]).size() == ((SelectResults) r[j][1]).size()) {
-        CacheUtils.log("Both SelectResults are of Same Size i.e.  Size= "
-            + ((SelectResults) r[j][1]).size());
-      } else {
-        fail("FAILED:SelectResults size is different in both the cases. Size1="
-            + ((SelectResults) r[j][0]).size() + " Size2 = "
-            + ((SelectResults) r[j][1]).size() + "; failed query=" + queries[j]);
-      }
+      checkSelectResultTypes((SelectResults)r[j][0], (SelectResults)r[j][1], queries[j]);
+      checkResultSizes((SelectResults)r[j][0], (SelectResults)r[j][1], queries[j]);
       
       if (checkOrder) {
         coll2 = (((SelectResults) r[j][1]).asList());
@@ -95,47 +75,7 @@ public class StructSetOrResultsSet extends TestCase {
         coll2 = (((SelectResults) r[j][1]).asSet());
         coll1 = (((SelectResults) r[j][0]).asSet());
       }
-      // boolean pass = true;
-      itert1 = coll1.iterator();
-      itert2 = coll2.iterator();
-      while (itert1.hasNext()) {
-        Object p1 = itert1.next();
-        if (!checkOrder) {
-          itert2 = coll2.iterator();
-        }
-
-        boolean exactMatch = false;
-        while (itert2.hasNext()) {
-          Object p2 = itert2.next();
-          if (p1 instanceof Struct) {
-            Object[] values1 = ((Struct) p1).getFieldValues();
-            Object[] values2 = ((Struct) p2).getFieldValues();
-            assertEquals(values1.length, values2.length);
-            boolean elementEqual = true;
-            for (int i = 0; i < values1.length; ++i) {
-              // CacheUtils.log("Comparing: " + values1[i] + " with: " +
-              // values2[i]);
-              elementEqual = elementEqual
-                  && ((values1[i] == values2[i]) || values1[i]
-                      .equals(values2[i]));
-            }
-            exactMatch = elementEqual;
-          } else {
-            // CacheUtils.log("Comparing: " + p1 + " with: " + p2);
-            exactMatch = (p2 == p1) || p2.equals(p1);
-          }
-          if (exactMatch || checkOrder) {
-            break;
-          }
-        }
-        if (!exactMatch) {
-          fail("Atleast one element in the pair of SelectResults "
-              + "supposedly identical, is not equal " + "Match not found for :"
-              + p1 + "; failed query=" + queries[j] + "; element unmatched ="
-              + p1 + ";p1 class=" + p1.getClass() + " ; other set has ="
-              + coll2);
-        }
-      }
+      compareResults(coll1, coll2, queries[j], checkOrder);
     }
   }
 
@@ -357,38 +297,10 @@ public class StructSetOrResultsSet extends TestCase {
       assertEquals(queries[j], 1, result1.size());
       assertEquals(queries[j], 1, result2.size());
 
-      if ((result1.asList().get(0).getClass().getName()).equals(result2
-          .asList().get(0).getClass().getName())) {
-        CacheUtils.log("Both SelectResults are of the same Type i.e.--> "
-            + ((SelectResults) r[j][0]).getCollectionType().getElementType());
-      } else {
-        fail("FAILED:Select result Type is different in both the cases."
-            + "; failed query=" + queries[j]);
-      }
-
-      if (((SelectResults) r[j][0]).size() == ((SelectResults) r[j][1]).size()) {
-        CacheUtils.log("Both SelectResults are of Same Size i.e.  Size= "
-            + ((SelectResults) r[j][1]).size());
-      } else {
-        fail("FAILED:SelectResults size is different in both the cases. Size1="
-            + ((SelectResults) r[j][0]).size() + " Size2 = "
-            + ((SelectResults) r[j][1]).size() + "; failed query=" + queries[j]);
-      }
-
-      // boolean pass = true;
-      itert1 = result1.iterator();
-      itert2 = result2.iterator();
-      while (itert1.hasNext()) {
-        Integer p1 = itert1.next();
-        Integer p2 = itert2.next();
-        CacheUtils.log("result1: " + p1 + "result2: " + p2);
-        exactMatch &= p1.intValue() == p2.intValue();
-
-      }
-      if (!exactMatch) {
-        fail("Atleast one element in the pair of SelectResults supposedly identical, is not equal "
-            + "; failed query=" + queries[j]);
-      }
+      
+      checkSelectResultTypes((SelectResults) r[j][0], (SelectResults) r[j][1], queries[j]);
+      checkResultSizes((SelectResults) r[j][0], (SelectResults) r[j][1], queries[j]);
+      compareResults(result1, result2, queries[j], true);
     }
   }
 
@@ -413,71 +325,86 @@ public class StructSetOrResultsSet extends TestCase {
     Integer count1, count2;
     Iterator<Integer> itert1, itert2;
     ArrayList result1, result2;
+
     for (int j = 0; j < len; j++) {
       result1 = ((ArrayList) r[j][0]);
       result2 = ((ArrayList) r[j][1]);
       result1.trimToSize();
       result2.trimToSize();
-      // assertFalse(queries[j], result1.size()==0);
-      // assertFalse(queries[j], result2.size()==0);
-
-      if (checkClass) {
-        if ((result1.get(0).getClass().getName()).equals(result2.get(0)
-            .getClass().getName())) {
-          CacheUtils.log("Both SelectResults are of the same Type i.e.--> "
-              + result1.get(0).getClass().getName());
-        } else {
-          fail("FAILED:Select result Type is different in both the cases."
-              + result1.get(0).getClass().getName() + "and"
-              + result1.get(0).getClass().getName() + "; failed query="
-              + queries[j]);
-        }
-      }
-
-      if (result1.size() == result2.size()) {
-        CacheUtils.log("Both SelectResults are of Same Size i.e.  Size= "
-            + result2.size());
+    
+      compareQueryResultLists(result1, result2, len, checkOrder, checkClass, queries[j]);
+    }
+  }
+  
+  public void compareQueryResultLists(List r1, List r2,
+      int len, boolean checkOrder, boolean checkClass, String query) {
+    if (checkClass) {
+      if ((r1.get(0).getClass().getName()).equals(r2.get(0)
+          .getClass().getName())) {
+        CacheUtils.log("Both SelectResults are of the same Type i.e.--> "
+            + r1.get(0).getClass().getName());
       } else {
-        fail("FAILED:SelectResults size is different in both the cases. Size1="
-            + result1.size() + " Size2 = " + result2.size() + "; failed query="
-            + queries[j]);
+        fail("FAILED:Select result Type is different in both the cases."
+            + r1.get(0).getClass().getName() + "and"
+            + r1.get(0).getClass().getName() + "; failed query="
+            + query);
       }
+    }
 
-      // boolean pass = true;
-      itert1 = result1.iterator();
-      itert2 = result2.iterator();
-      while (itert1.hasNext()) {
-        Object p1 = itert1.next();
-        if (!checkOrder) {
-          itert2 = result2.iterator();
+    checkResultSizes(r1, r2, query);
+    compareResults(r1, r2, query, checkOrder);
+  }
+  
+  private void checkSelectResultTypes(SelectResults r1, SelectResults r2, String query) {
+    ObjectType type1, type2;
+    type1 = r1.getCollectionType().getElementType();
+    type2 = r2.getCollectionType().getElementType();
+    if (!(type1.getClass().getName()).equals(type2.getClass().getName())) {
+      CacheUtils.log("Classes are : " + type1.getClass().getName() + " " + type2.getClass().getName());
+      fail("FAILED:Select result Type is different in both the cases." + "; failed query=" + query);
+    }
+  }
+  
+  private void checkResultSizes(Collection r1, Collection r2, String query) {
+    if (r1.size() != r2.size()) {
+      fail("FAILED:SelectResults size is different in both the cases. Size1="
+          + ((SelectResults) r1).size() + " Size2 = "
+          + ((SelectResults) r2).size() + "; failed query=" + query);
+    }
+  }
+  
+  
+  private void compareResults(Collection result1, Collection result2, String query, boolean checkOrder) {
+    Iterator itert1 = result1.iterator();
+    Iterator itert2 = result2.iterator();
+    int currentIndex = 0;
+    while (itert1.hasNext()) {
+      Object p1 = itert1.next();
+      Object p2 = null;
+      if (!checkOrder) {
+        if (!result2.contains(p1)) {
+          fail("Atleast one element in the pair of SelectResults "
+              + "supposedly identical, is not equal " + "Match not found for :"
+              + p1 + " compared with:" + p2 + "; failed query=" + query + "; element unmatched ="
+              + p1 + ";p1 class=" + p1.getClass() + " ; other set has ="
+              + result2);
         }
-
-        boolean exactMatch = false;
-        while (itert2.hasNext()) {
-          Object p2 = itert2.next();
-          if (p1 instanceof Struct) {
-            Object[] values1 = ((Struct) p1).getFieldValues();
-            Object[] values2 = ((Struct) p2).getFieldValues();
-            assertEquals(values1.length, values2.length);
-            boolean elementEqual = true;
-            for (int i = 0; i < values1.length; ++i) {
-              elementEqual = elementEqual
-                  && ((values1[i] == values2[i]) || values1[i]
-                      .equals(values2[i]));
-            }
-            exactMatch = elementEqual;
-          } else {
-            exactMatch = (p2 == p1) || p2.equals(p1);
-          }
-          if (exactMatch || checkOrder) {
-            break;
+      }
+      else {
+        boolean matched = false;
+        if (itert2.hasNext()) {
+          p2 = itert2.next();
+          matched = (p2 == p1) || p2.equals(p1);
+          if (!matched) {
+            fail("Order of results was not the same, match not found for :"
+                + p1 + " compared with:" + p2 + "; failed query=" + query + "; element unmatched ="
+                + p1 + ";p1 class=" + p1.getClass() + " compared with " + p2 + ";p2 class=" + p2.getClass()
+                + "currentIndex:" + currentIndex 
+                + "\nr1:" + result1 + "\n\nr2:" + result2);
           }
         }
-        if (!exactMatch) {
-          fail("Atleast one element in the pair of SelectResults supposedly identical, is not equal "
-              + "; failed query=" + queries[j]);
-        }
       }
+      currentIndex ++;
     }
   }
 }


[06/38] incubator-geode git commit: [GEODE-146] Queries are not thread safe due to scopeId variable ScopeId was not thread safe when stored and used in CompiledSelect Multiple threads could set/unset the value. Instead it is stored in the query executio

Posted by bs...@apache.org.
[GEODE-146] Queries are not thread safe due to scopeId variable
ScopeId was not thread safe when stored and used in CompiledSelect
Multiple threads could set/unset the value.  Instead it is stored in
the query execution context.


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

Branch: refs/heads/feature/GEODE-77
Commit: 1512677f721279a05c6ebddad443d2043d032f99
Parents: fc9ee58
Author: Jason Huynh <jh...@pivotal.io>
Authored: Mon Oct 5 09:58:04 2015 -0700
Committer: Jason Huynh <jh...@pivotal.io>
Committed: Mon Oct 5 10:01:57 2015 -0700

----------------------------------------------------------------------
 .../cache/query/internal/CompiledSelect.java    | 34 +++-----
 .../gemfire/cache/query/QueryJUnitTest.java     | 91 ++++++++++++++++++++
 2 files changed, 103 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1512677f/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledSelect.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledSelect.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledSelect.java
index 0b07211..a18d9c3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledSelect.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/CompiledSelect.java
@@ -59,7 +59,6 @@ public class CompiledSelect extends AbstractCompiledValue {
      // 0 is projection name, 1 is the CompiledValue for the expression 
   private boolean distinct;
   private boolean count;
-  private int scopeID;
   //Asif: limits the SelectResults by the number specified.
   private CompiledValue limit;
   //Shobhit: counts the no of results satisfying where condition for
@@ -72,17 +71,10 @@ public class CompiledSelect extends AbstractCompiledValue {
   protected boolean transformationDone = false;
   protected ObjectType cachedElementTypeForOrderBy = null;
   private boolean hasUnmappedOrderByCols = false; 
-  
 
-  /** 
-   * Identifies the scope ID assosciated with the Select. The CompiledSelect object
-   * is shared across by multiple query executing threads, but since the scopeID 
-   * which gets assigned in the computeDependency phase & is obtained from 
-   * ExecutionContext, it will not differ across threads. This field may get reassigned
-   * by various threads, but still the value will be consistent.
-   * It is also therefore not needed to make this field volatile
-   * Asif
-   */
+  //used as a key in a context to identify the scope of this CompiledSelect 
+  private Object scopeID = new Object(); 
+
   public CompiledSelect(boolean distinct, boolean count, CompiledValue whereClause,
                         List iterators, List projAttrs,List<CompiledSortCriterion> orderByAttrs, CompiledValue limit,
                         List<String> hints, List<CompiledValue> groupByClause) {
@@ -170,10 +162,9 @@ public class CompiledSelect extends AbstractCompiledValue {
          AmbiguousNameException,
          NameResolutionException {
     // bind iterators in new scope in order to determine dependencies
-    //int scopeID = context.assosciateScopeID(this);
-    this.scopeID = context.assosciateScopeID();
-    context.newScope(scopeID);
-    context.pushExecCache(scopeID);
+    context.cachePut(scopeID, context.assosciateScopeID());
+    context.newScope((Integer)context.cacheGet(scopeID));
+    context.pushExecCache((Integer)context.cacheGet(scopeID));
     try {
       Iterator iter = this.iterators.iterator();
       while (iter.hasNext()) {
@@ -365,8 +356,8 @@ public class CompiledSelect extends AbstractCompiledValue {
   throws FunctionDomainException, TypeMismatchException, NameResolutionException, QueryInvocationTargetException {
     ExecutionContext context = new QueryExecutionContext(parameters, cache, query);
     computeDependencies(context);
-    context.newScope(this.scopeID);
-    context.pushExecCache(scopeID);
+    context.newScope((Integer)context.cacheGet(scopeID));
+    context.pushExecCache((Integer)context.cacheGet(scopeID));
     SelectResults results = null;
     try {
       Iterator iter = iterators.iterator();
@@ -390,9 +381,8 @@ public class CompiledSelect extends AbstractCompiledValue {
   
   public SelectResults evaluate(ExecutionContext context) throws FunctionDomainException, TypeMismatchException,
       NameResolutionException, QueryInvocationTargetException {
-   // context.newScope(context.getScopeID(this));
-    context.newScope(this.scopeID);
-    context.pushExecCache(scopeID);
+    context.newScope((Integer)context.cacheGet(scopeID));
+    context.pushExecCache((Integer)context.cacheGet(scopeID));
     context.setDistinct(this.distinct);
     if(this.hasUnmappedOrderByCols && context.getBucketList() != null) {
       throw new QueryInvalidException(LocalizedStrings.DefaultQuery_ORDER_BY_ATTRIBS_NOT_PRESENT_IN_PROJ.toLocalizedString()); 
@@ -1477,8 +1467,8 @@ public class CompiledSelect extends AbstractCompiledValue {
       return true;
     }
 
-    context.newScope(this.scopeID);
-    context.pushExecCache(scopeID);
+    context.newScope((Integer)context.cacheGet(scopeID));
+    context.pushExecCache((Integer)context.cacheGet(scopeID));
     try {
       CompiledIteratorDef iterDef = (CompiledIteratorDef) iterators.get(0);
       RuntimeIterator rIter = iterDef.getRuntimeIterator(context);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1512677f/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java
index 5490278..03b6d41 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java
@@ -23,7 +23,11 @@ import static org.junit.runners.MethodSorters.NAME_ASCENDING;
 
 import java.util.Arrays;
 import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Set;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.TimeUnit;
 
 import org.junit.After;
 import org.junit.Before;
@@ -31,9 +35,14 @@ import org.junit.FixMethodOrder;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import util.TestException;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.PartitionAttributesFactory;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
@@ -353,5 +362,87 @@ public class QueryJUnitTest {
     }
     assertEquals("Incorrect result size ", 1, sr.size());
   }
+
+  @Test
+  public void testThreadSafetyOfCompiledSelectScopeId() throws Exception {
+    try {
+      Cache cache = CacheUtils.getCache();
+      RegionFactory<Integer, Portfolio> rf = cache
+          .createRegionFactory(RegionShortcut.PARTITION);
+      Region r = rf.create("keyzset");
+      for (int i = 0; i < 100; i++) {
+        r.put(i, new Portfolio(i));
+      }
+      ScopeThreadingTestHook scopeIDTestHook = new ScopeThreadingTestHook(3);
+      DefaultQuery.testHook = scopeIDTestHook;
+      QueryService qs = cache.getQueryService();
+      Query q = qs
+          .newQuery("SELECT DISTINCT * FROM /keyzset.keySet key WHERE key.id > 0 AND key.id <= 0 ORDER BY key asc LIMIT $3");
+      Thread q1 = new Thread(new QueryRunnable(q, new Object[] { 10, 20, 10 }));
+      Thread q2 = new Thread(new QueryRunnable(q, new Object[] { 5, 10, 5 }));
+      Thread q3 = new Thread(new QueryRunnable(q, new Object[] { 2, 10, 8 }));
+      q1.start();
+      q2.start();
+      q3.start();
+      q1.join();
+      q2.join();
+      q3.join();
+      assertEquals("Exceptions were thrown due to DefaultQuery not being thread-safe", true, scopeIDTestHook.isOk());
+    }
+    finally {
+      DefaultQuery.testHook = null;
+    }
+  }
+
+  private class QueryRunnable implements Runnable {
+    private Query q;
+    private Object[] params;
+
+    public QueryRunnable(Query q, Object[] params) {
+      this.q = q;
+      this.params = params;
+    }
+
+    public void run() {
+      try {
+        q.execute(params);
+      } catch (Exception e) {
+        throw new TestException("exception occured while executing query", e);
+      }
+    }
+  }
+
+  public class ScopeThreadingTestHook implements DefaultQuery.TestHook {
+    private CyclicBarrier barrier;
+    private List<Exception> exceptionsThrown = new LinkedList<Exception>();
+
+    public ScopeThreadingTestHook(int numThreads) {
+      barrier = new CyclicBarrier(numThreads);
+    }
+
+    @Override
+    public void doTestHook(int spot) {
+      this.doTestHook(spot + "");
+    }
+
+    @Override
+    public void doTestHook(String spot) {
+      if (spot.equals("1")) {
+        try {
+          barrier.await(8, TimeUnit.SECONDS);
+        } catch (InterruptedException e) {
+          exceptionsThrown.add(e);
+          Thread.currentThread().interrupt();
+        } catch (Exception e) {
+          exceptionsThrown.add(e);
+        }
+      }
+    }
+
+    public boolean isOk() {
+      return exceptionsThrown.size() == 0;
+    }
+  }
+
   
 }


[34/38] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-77

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
index 1a50e1c,58ef0ab..6b10ea1
--- a/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
@@@ -218,17 -218,8 +218,7 @@@ com/gemstone/gemfire/cache/query/intern
  com/gemstone/gemfire/cache/query/internal/utils/PDXUtils$2,false
  com/gemstone/gemfire/cache/snapshot/SnapshotOptions$SnapshotFormat,false
  com/gemstone/gemfire/cache/util/BoundedLinkedHashMap,true,-3419897166186852692,_maximumNumberOfEntries:int
- com/gemstone/gemfire/cache/util/BridgeWriterException,true,-295001316745954159
- com/gemstone/gemfire/cache/util/EndpointDoesNotExistException,true,1654241470788247283
- com/gemstone/gemfire/cache/util/EndpointException,false
- com/gemstone/gemfire/cache/util/EndpointExistsException,true,950617116786308012
- com/gemstone/gemfire/cache/util/EndpointInUseException,true,-4087729485272321469
- com/gemstone/gemfire/cache/util/Gateway$OrderPolicy,false
- com/gemstone/gemfire/cache/util/IncompatibleVersionException,true,7008667865037538081
- com/gemstone/gemfire/cache/util/ServerRefusedConnectionException,true,-4996327025772566931
- com/gemstone/gemfire/cache/util/UnknownVersionException,true,7379530185697556990
- com/gemstone/gemfire/cache/util/VersionException,false
  com/gemstone/gemfire/cache/wan/GatewaySender$OrderPolicy,false
 -com/gemstone/gemfire/cache/util/Gateway$OrderPolicy,false
  com/gemstone/gemfire/compression/CompressionException,true,4118639654597191235
  com/gemstone/gemfire/compression/SnappyCompressor,true,496609875302446099
  com/gemstone/gemfire/distributed/AbstractLauncher$Status,false,description:java/lang/String


[35/38] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-77

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/resources/log4j2.xml
----------------------------------------------------------------------
diff --cc gemfire-core/src/main/resources/log4j2.xml
index 0000000,a21a333..00a1d91
mode 000000,100755..100755
--- a/gemfire-core/src/main/resources/log4j2.xml
+++ b/gemfire-core/src/main/resources/log4j2.xml
@@@ -1,0 -1,22 +1,23 @@@
+ <?xml version="1.0" encoding="UTF-8"?>
+ <Configuration status="FATAL" shutdownHook="disable" packages="com.gemstone.gemfire.internal.logging.log4j">
+   <Properties>
+     <Property name="gemfire-pattern">[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} &lt;%thread&gt; tid=%tid] %message%n%throwable%n</Property>
+     <Property name="gemfire-default">true</Property>
+   </Properties>
+   <Appenders>
+     <Console name="STDOUT" target="SYSTEM_OUT">
+       <PatternLayout pattern="${gemfire-pattern}"/>
+     </Console>
+   </Appenders>
+   <Loggers>
+     <Logger name="com.gemstone" level="INFO" additivity="true">
+     <filters>
+       <MarkerFilter marker="GEMFIRE_VERBOSE" onMatch="DENY" onMismatch="NEUTRAL"/>
+     </filters>
+     </Logger>
++    <Logger name="org.jgroups" level="WARN" additivity="true"/>
+     <Root level="INFO">
+       <AppenderRef ref="STDOUT"/>
+     </Root>
+   </Loggers>
+ </Configuration>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
index 0000000,5dd3cfb..5f3de0e
mode 000000,100644..100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
@@@ -1,0 -1,1642 +1,1644 @@@
+ /*=========================================================================
+  * 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.net.InetAddress;
+ import java.net.InetSocketAddress;
+ import java.net.Socket;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Properties;
+ import java.util.Set;
+ 
+ import com.gemstone.gemfire.InternalGemFireException;
+ import com.gemstone.gemfire.LogWriter;
+ import com.gemstone.gemfire.Statistics;
+ import com.gemstone.gemfire.StatisticsType;
+ import com.gemstone.gemfire.cache.AttributesFactory;
+ import com.gemstone.gemfire.cache.CacheException;
+ import com.gemstone.gemfire.cache.Region;
+ import com.gemstone.gemfire.cache.Scope;
+ import com.gemstone.gemfire.cache.client.ClientCache;
+ import com.gemstone.gemfire.cache.client.Pool;
+ import com.gemstone.gemfire.cache.client.PoolManager;
+ import com.gemstone.gemfire.distributed.DistributedMember;
+ import com.gemstone.gemfire.distributed.DurableClientAttributes;
+ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+ import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+ 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.ServerConnection;
+ import com.gemstone.gemfire.internal.logging.LocalLogWriter;
+ import com.gemstone.gemfire.internal.logging.InternalLogWriter;
+ import com.gemstone.gemfire.management.membership.ClientMembership;
+ import com.gemstone.gemfire.management.membership.ClientMembershipEvent;
+ import com.gemstone.gemfire.management.membership.ClientMembershipListener;
+ 
+ import dunit.DistributedTestCase;
+ import dunit.Host;
+ import dunit.SerializableRunnable;
+ import dunit.VM;
+ import dunit.DistributedTestCase.WaitCriterion;
+ 
+ /**
+  * Tests the ClientMembership API including ClientMembershipListener.
+  *
+  * @author Kirk Lund
+  * @since 4.2.1
+  */
+ public class ClientMembershipDUnitTest extends ClientServerTestCase {
+ 
+   protected static final boolean CLIENT = true;
+   protected static final boolean SERVER = false;
+   
+   protected static final int JOINED = 0;
+   protected static final int LEFT = 1;
+   protected static final int CRASHED = 2;
+     
+   public ClientMembershipDUnitTest(String name) {
+     super(name);
+   }
+ 
+   public void setUp() throws Exception {
+     super.setUp();
+     getSystem();
+   }
+   
+   public void tearDown2() throws Exception {
+     super.tearDown2();
+     InternalClientMembership.unregisterAllListeners();
+   }
+ 
+   private void waitForAcceptsInProgressToBe(final int target)
+     throws Exception {
+     WaitCriterion ev = new WaitCriterion() {
+       String excuse;
+       public boolean done() {
+         int actual = getAcceptsInProgress();
+         if (actual == getAcceptsInProgress()) {
+           return true;
+         }
+         excuse = "accepts in progress (" + actual + ") never became " + target;
+         return false;
+       }
+       public String description() {
+         return excuse;
+       }
+     };
+     DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+   }
+   
+   protected int getAcceptsInProgress() {
+     StatisticsType st = InternalDistributedSystem.getAnyInstance().findType("CacheServerStats");
+     Statistics[] s = InternalDistributedSystem.getAnyInstance().findStatisticsByType(st);
+     return s[0].getInt("acceptsInProgress");
+   }
+ 
+   protected static Socket meanSocket;
+ 
+   /** test that a server times out waiting for a handshake that
+       never arrives. 
+    */
+   public void testConnectionTimeout() throws Exception {
+     addExpectedException("failed accepting client connection");
+     final Host host = Host.getHost(0);
+     final String hostName = getServerHostName(host);
+     final VM vm0 = host.getVM(0);
+     System.setProperty(AcceptorImpl.ACCEPT_TIMEOUT_PROPERTY_NAME, "1000");
+     try {
+     final int port = startBridgeServer(0);
+ //    AsyncInvocation ai = null;
+     try {
+       assertTrue(port != 0);
+       SerializableRunnable createMeanSocket = new CacheSerializableRunnable("Connect to server with socket") {
+         public void run2() throws CacheException {
+           getCache(); // create a cache so we have stats
+           getLogWriter().info("connecting to cache server with socket");
+           try {
+             InetAddress addr = InetAddress.getByName(hostName);
+             meanSocket = new Socket(addr, port);
+           }
+           catch (Exception e) {
+             throw new RuntimeException("Test failed to connect or was interrupted", e);
+           }
+         }
+       };
+       SerializableRunnable closeMeanSocket = new CacheSerializableRunnable("close mean socket") {
+         public void run2() throws CacheException {
+           getLogWriter().info("closing mean socket");
+           try {
+             meanSocket.close();
+           }
+           catch (IOException ignore) {
+           }
+         }
+       };
+ 
+       assertEquals(0, getAcceptsInProgress());
+       
+       getLogWriter().info("creating mean socket");
+       vm0.invoke(createMeanSocket);
+       try {
+         getLogWriter().info("waiting to see it connect on server");
+         waitForAcceptsInProgressToBe(1);
+       } finally {
+         getLogWriter().info("closing mean socket");
+         vm0.invoke(closeMeanSocket);
+       }
+       getLogWriter().info("waiting to see accept to go away on server");
+       waitForAcceptsInProgressToBe(0);
+ 
+       // now try it without a close. Server should timeout the mean connect
+       getLogWriter().info("creating mean socket 2");
+       vm0.invoke(createMeanSocket);
+       try {
+         getLogWriter().info("waiting to see it connect on server 2");
+         waitForAcceptsInProgressToBe(1);
+         getLogWriter().info("waiting to see accept to go away on server without us closing");
+         waitForAcceptsInProgressToBe(0);
+       } finally {
+         getLogWriter().info("closing mean socket 2");
+         vm0.invoke(closeMeanSocket);
+       }
+ 
+ //       SerializableRunnable denialOfService = new CacheSerializableRunnable("Do lots of connects") {
+ //         public void run2() throws CacheException {
+ //           int connectionCount = 0;
+ //           ArrayList al = new ArrayList(60000);
+ //           try {
+ //             InetAddress addr = InetAddress.getLocalHost();
+ //             for (;;) {
+ //               Socket s = new Socket(addr, port);
+ //               al.add(s);
+ //               connectionCount++;
+ //               getLogWriter().info("connected # " + connectionCount + " s=" + s);
+ // //               try {
+ // //                 s.close();
+ // //               } catch (IOException ignore) {}
+ //             }
+ //           }
+ //           catch (Exception e) {
+ //             getLogWriter().info("connected # " + connectionCount
+ //                                 + " stopped because of exception " + e);
+ //             Iterator it = al.iterator();
+ //             while (it.hasNext()) {
+ //               Socket s = (Socket)it.next();
+ //               try {
+ //                 s.close();
+ //               } catch (IOException ignore) {}
+ //             }
+ //           }
+ //         }
+ //       };
+ //       // now pretend to do a denial of service attack by doing a bunch of connects
+ //       // really fast and see what that does to the server's fds.
+ //       getLogWriter().info("doing denial of service attach");
+ //       vm0.invoke(denialOfService);
+ //       // @todo darrel: check fd limit?
+     }
+     finally {
+       stopBridgeServers(getCache());
+     }
+     }
+     finally {
+       System.getProperties().remove(AcceptorImpl.ACCEPT_TIMEOUT_PROPERTY_NAME);
+     }
+   }
+ 
+   public void testSynchronousEvents() throws Exception {
+     InternalClientMembership.setForceSynchronous(true);
+     try {
+       doTestBasicEvents();
+     }
+     finally {
+       InternalClientMembership.setForceSynchronous(false);
+     }
+   }
+   
+   /**
+    * Tests event notification methods on ClientMembership.
+    */
+   public void testBasicEvents() throws Exception {
+     doTestBasicEvents();
+   }
+   
+   public void doTestBasicEvents() throws Exception {
+     final boolean[] fired = new boolean[3];
+     final DistributedMember[] member = new DistributedMember[3];
+     final String[] memberId = new String[3];
+     final boolean[] isClient = new boolean[3];
+     
+     ClientMembershipListener listener = new ClientMembershipListener() {
+       public synchronized void memberJoined(ClientMembershipEvent event) {
+         fired[JOINED] = true;
+         member[JOINED] = event.getMember();
+         memberId[JOINED] = event.getMemberId();
+         isClient[JOINED] = event.isClient();
+         notify();
+       }
+       public synchronized void memberLeft(ClientMembershipEvent event) {
+         fired[LEFT] = true;
+         member[LEFT] = event.getMember();
+         memberId[LEFT] = event.getMemberId();
+         isClient[LEFT] = event.isClient();
+         notify();
+       }
+       public synchronized void memberCrashed(ClientMembershipEvent event) {
+         fired[CRASHED] = true;
+         member[CRASHED] = event.getMember();
+         memberId[CRASHED] = event.getMemberId();
+         isClient[CRASHED] = event.isClient();
+         notify();
+       }
+     };
+     ClientMembership.registerClientMembershipListener(listener);
+     
+     // test JOIN for server
+     DistributedMember serverJoined = new TestDistributedMember("serverJoined");
+     InternalClientMembership.notifyJoined(serverJoined, SERVER);
+     synchronized(listener) {
+       if (!fired[JOINED]) {
+         listener.wait(2000);
+       }
+     }
+     assertTrue(fired[JOINED]);
+     assertEquals(serverJoined, member[JOINED]);
+     assertEquals(serverJoined.getId(), memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     // test JOIN for client
+     DistributedMember clientJoined = new TestDistributedMember("clientJoined");
+     InternalClientMembership.notifyJoined(clientJoined, CLIENT);
+     synchronized(listener) {
+       if (!fired[JOINED]) {
+         listener.wait(2000);
+       }
+     }
+     assertTrue(fired[JOINED]);
+     assertEquals(clientJoined, member[JOINED]);
+     assertEquals(clientJoined.getId(), memberId[JOINED]);
+     assertTrue(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     // test LEFT for server
+     DistributedMember serverLeft = new TestDistributedMember("serverLeft");
+     InternalClientMembership.notifyLeft(serverLeft, SERVER);
+     synchronized(listener) {
+       if (!fired[LEFT]) {
+         listener.wait(2000);
+       }
+     }
+     assertFalse(fired[JOINED]);
+     assertNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertTrue(fired[LEFT]);
+     assertEquals(serverLeft, member[LEFT]);
+     assertEquals(serverLeft.getId(), memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     // test LEFT for client
+     DistributedMember clientLeft = new TestDistributedMember("clientLeft");
+     InternalClientMembership.notifyLeft(clientLeft, CLIENT);
+     synchronized(listener) {
+       if (!fired[LEFT]) {
+         listener.wait(2000);
+       }
+     }
+     assertFalse(fired[JOINED]);
+     assertNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertTrue(fired[LEFT]);
+     assertEquals(clientLeft, member[LEFT]);
+     assertEquals(clientLeft.getId(), memberId[LEFT]);
+     assertTrue(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     // test CRASHED for server
+     DistributedMember serverCrashed = new TestDistributedMember("serverCrashed");
+     InternalClientMembership.notifyCrashed(serverCrashed, SERVER);
+     synchronized(listener) {
+       if (!fired[CRASHED]) {
+         listener.wait(2000);
+       }
+     }
+     assertFalse(fired[JOINED]);
+     assertNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertTrue(fired[CRASHED]);
+     assertEquals(serverCrashed, member[CRASHED]);
+     assertEquals(serverCrashed.getId(), memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     // test CRASHED for client
+     DistributedMember clientCrashed = new TestDistributedMember("clientCrashed");
+     InternalClientMembership.notifyCrashed(clientCrashed, CLIENT);
+     synchronized(listener) {
+       if (!fired[CRASHED]) {
+         listener.wait(2000);
+       }
+     }
+     assertFalse(fired[JOINED]);
+     assertNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertTrue(fired[CRASHED]);
+     assertEquals(clientCrashed, member[CRASHED]);
+     assertEquals(clientCrashed.getId(), memberId[CRASHED]);
+     assertTrue(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+   }
+   
+   /**
+    * Resets all elements of arrays used for listener testing. Boolean values
+    * are reset to false. String values are reset to null.
+    */
+   private void resetArraysForTesting(boolean[] fired, 
+                                      DistributedMember[] member,
+                                      String[] memberId, 
+                                      boolean[] isClient) {
+     for (int i = 0; i < fired.length; i++) {
+       fired[i] = false;
+       member[i] = null;
+       memberId[i] = null;
+       isClient[i] = false;
+     }
+   }
+   
+   /**
+    * Tests unregisterClientMembershipListener to ensure that no further events
+    * are delivered to unregistered listeners.
+    */
+   public void testUnregisterClientMembershipListener() throws Exception {
+     final boolean[] fired = new boolean[1];
+     final DistributedMember[] member = new DistributedMember[1];
+     final String[] memberId = new String[1];
+     final boolean[] isClient = new boolean[1];
+     
+     ClientMembershipListener listener = new ClientMembershipListener() {
+       public synchronized void memberJoined(ClientMembershipEvent event) {
+         fired[0] = true;
+         member[0] = event.getMember();
+         memberId[0] = event.getMemberId();
+         isClient[0] = event.isClient();
+         notify();
+       }
+       public void memberLeft(ClientMembershipEvent event) {
+       }
+       public void memberCrashed(ClientMembershipEvent event) {
+       }
+     };
+     ClientMembership.registerClientMembershipListener(listener);
+     
+     // fire event to make sure listener is registered
+     DistributedMember clientJoined = new TestDistributedMember("clientJoined");
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     synchronized(listener) {
+       if (!fired[0]) {
+         listener.wait(2000);
+       }
+     }
+     assertTrue(fired[0]);
+     assertEquals(clientJoined, member[0]);
+     assertEquals(clientJoined.getId(), memberId[0]);
+     assertTrue(isClient[0]);
+ 
+     resetArraysForTesting(fired, member, memberId, isClient);
+     assertFalse(fired[0]);
+     assertNull(memberId[0]);
+     assertFalse(isClient[0]);
+ 
+     // unregister and verify listener is not notified
+     ClientMembership.unregisterClientMembershipListener(listener);
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     synchronized(listener) {
+       listener.wait(20);
+     }
+     assertFalse(fired[0]);
+     assertNull(member[0]);
+     assertNull(memberId[0]);
+     assertFalse(isClient[0]);
+   }
+   
+   public void testMultipleListeners() throws Exception {
+     final int NUM_LISTENERS = 4;
+     final boolean[] fired = new boolean[NUM_LISTENERS];
+     final DistributedMember[] member = new DistributedMember[NUM_LISTENERS];
+     final String[] memberId = new String[NUM_LISTENERS];
+     final boolean[] isClient = new boolean[NUM_LISTENERS];
+     
+     final ClientMembershipListener[] listeners = new ClientMembershipListener[NUM_LISTENERS];
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       final int whichListener = i;
+       listeners[i] = new ClientMembershipListener() {
+         public synchronized void memberJoined(ClientMembershipEvent event) {
+           assertFalse(fired[whichListener]);
+           assertNull(member[whichListener]);
+           assertNull(memberId[whichListener]);
+           assertFalse(isClient[whichListener]);
+           fired[whichListener] = true;
+           member[whichListener] = event.getMember();
+           memberId[whichListener] = event.getMemberId();
+           isClient[whichListener] = event.isClient();
+           notify();
+         }
+         public void memberLeft(ClientMembershipEvent event) {
+         }
+         public void memberCrashed(ClientMembershipEvent event) {
+         }
+       };
+     }
+     
+     final DistributedMember clientJoined = new TestDistributedMember("clientJoined");
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       synchronized(listeners[i]) {
+         listeners[i].wait(20);
+       }
+       assertFalse(fired[i]);
+       assertNull(member[i]);
+       assertNull(memberId[i]);
+       assertFalse(isClient[i]);
+     }
+     
+     // attempt to register same listener twice... 2nd reg should be ignored
+     // failure would cause an assertion failure in memberJoined impl
+     ClientMembership.registerClientMembershipListener(listeners[0]);
+     ClientMembership.registerClientMembershipListener(listeners[0]);
+     
+     ClientMembershipListener[] registeredListeners = 
+       ClientMembership.getClientMembershipListeners();
+     assertEquals(1, registeredListeners.length);
+     assertEquals(listeners[0], registeredListeners[0]);
+     
+     ClientMembership.registerClientMembershipListener(listeners[1]);
+     registeredListeners = ClientMembership.getClientMembershipListeners();
+     assertEquals(2, registeredListeners.length);
+     assertEquals(listeners[0], registeredListeners[0]);
+     assertEquals(listeners[1], registeredListeners[1]);
+ 
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     synchronized(listeners[1]) {
+       if (!fired[1]) {
+         listeners[1].wait(2000);
+       }
+     }
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       if (i < 2) {
+         assertTrue(fired[i]);
+         assertEquals(clientJoined, member[i]);
+         assertEquals(clientJoined.getId(), memberId[i]);
+         assertTrue(isClient[i]);
+       } else {
+         assertFalse(fired[i]);
+         assertNull(member[i]);
+         assertNull(memberId[i]);
+         assertFalse(isClient[i]);
+       }
+     }
+     resetArraysForTesting(fired, member, memberId, isClient);
+         
+     ClientMembership.unregisterClientMembershipListener(listeners[0]);
+     registeredListeners = ClientMembership.getClientMembershipListeners();
+     assertEquals(1, registeredListeners.length);
+     assertEquals(listeners[1], registeredListeners[0]);
+     
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     synchronized(listeners[1]) {
+       if (!fired[1]) {
+         listeners[1].wait(2000);
+       }
+     }
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       if (i == 1) {
+         assertTrue(fired[i]);
+         assertEquals(clientJoined, member[i]);
+         assertEquals(clientJoined.getId(), memberId[i]);
+         assertTrue(isClient[i]);
+       } else {
+         assertFalse(fired[i]);
+         assertNull(member[i]);
+         assertNull(memberId[i]);
+         assertFalse(isClient[i]);
+       }
+     }
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     ClientMembership.registerClientMembershipListener(listeners[2]);
+     ClientMembership.registerClientMembershipListener(listeners[3]);
+     registeredListeners = ClientMembership.getClientMembershipListeners();
+     assertEquals(3, registeredListeners.length);
+     assertEquals(listeners[1], registeredListeners[0]);
+     assertEquals(listeners[2], registeredListeners[1]);
+     assertEquals(listeners[3], registeredListeners[2]);
+ 
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     synchronized(listeners[3]) {
+       if (!fired[3]) {
+         listeners[3].wait(2000);
+       }
+     }
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       if (i != 0) {
+         assertTrue(fired[i]);
+         assertEquals(clientJoined, member[i]);
+         assertEquals(clientJoined.getId(), memberId[i]);
+         assertTrue(isClient[i]);
+       } else {
+         assertFalse(fired[i]);
+         assertNull(member[i]);
+         assertNull(memberId[i]);
+         assertFalse(isClient[i]);
+       }
+     }
+     resetArraysForTesting(fired, member, memberId, isClient);
+     
+     ClientMembership.registerClientMembershipListener(listeners[0]);
+     registeredListeners = ClientMembership.getClientMembershipListeners();
+     assertEquals(4, registeredListeners.length);
+     assertEquals(listeners[1], registeredListeners[0]);
+     assertEquals(listeners[2], registeredListeners[1]);
+     assertEquals(listeners[3], registeredListeners[2]);
+     assertEquals(listeners[0], registeredListeners[3]);
+ 
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     synchronized(listeners[0]) {
+       if (!fired[0]) {
+         listeners[0].wait(2000);
+       }
+     }
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       assertTrue(fired[i]);
+       assertEquals(clientJoined, member[i]);
+       assertEquals(clientJoined.getId(), memberId[i]);
+       assertTrue(isClient[i]);
+     }
+     resetArraysForTesting(fired, member, memberId, isClient);
+     
+     ClientMembership.unregisterClientMembershipListener(listeners[3]);
+     registeredListeners = ClientMembership.getClientMembershipListeners();
+     assertEquals(3, registeredListeners.length);
+     assertEquals(listeners[1], registeredListeners[0]);
+     assertEquals(listeners[2], registeredListeners[1]);
+     assertEquals(listeners[0], registeredListeners[2]);
+     
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     synchronized(listeners[0]) {
+       if (!fired[0]) {
+         listeners[0].wait(2000);
+       }
+     }
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       if (i < 3) {
+         assertTrue(fired[i]);
+         assertEquals(clientJoined, member[i]);
+         assertEquals(clientJoined.getId(), memberId[i]);
+         assertTrue(isClient[i]);
+       } else {
+         assertFalse(fired[i]);
+         assertNull(member[i]);
+         assertNull(memberId[i]);
+         assertFalse(isClient[i]);
+       }
+     }
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     ClientMembership.unregisterClientMembershipListener(listeners[2]);
+     registeredListeners = ClientMembership.getClientMembershipListeners();
+     assertEquals(2, registeredListeners.length);
+     assertEquals(listeners[1], registeredListeners[0]);
+     assertEquals(listeners[0], registeredListeners[1]);
+     
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     synchronized(listeners[0]) {
+       if (!fired[0]) {
+         listeners[0].wait(2000);
+       }
+     }
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       if (i < 2) {
+         assertTrue(fired[i]);
+         assertEquals(clientJoined, member[i]);
+         assertEquals(clientJoined.getId(), memberId[i]);
+         assertTrue(isClient[i]);
+       } else {
+         assertFalse(fired[i]);
+         assertNull(member[i]);
+         assertNull(memberId[i]);
+         assertFalse(isClient[i]);
+       }
+     }
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     ClientMembership.unregisterClientMembershipListener(listeners[1]);
+     ClientMembership.unregisterClientMembershipListener(listeners[0]);
+     registeredListeners = ClientMembership.getClientMembershipListeners();
+     assertEquals(0, registeredListeners.length);
+     
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       synchronized(listeners[i]) {
+         listeners[i].wait(20);
+       }
+       assertFalse(fired[i]);
+       assertNull(member[i]);
+       assertNull(memberId[i]);
+       assertFalse(isClient[i]);
+     }
+     resetArraysForTesting(fired, member, memberId, isClient);
+     
+     ClientMembership.registerClientMembershipListener(listeners[1]);
+     registeredListeners = ClientMembership.getClientMembershipListeners();
+     assertEquals(1, registeredListeners.length);
+     assertEquals(listeners[1], registeredListeners[0]);
+     
+     InternalClientMembership.notifyJoined(clientJoined, true);
+     synchronized(listeners[1]) {
+       if (!fired[1]) {
+         listeners[1].wait(2000);
+       }
+     }
+     for (int i = 0; i < NUM_LISTENERS; i++) {
+       if (i == 1) {
+         assertTrue(fired[i]);
+         assertEquals(clientJoined, member[i]);
+         assertEquals(clientJoined.getId(), memberId[i]);
+         assertTrue(isClient[i]);
+       } else {
+         assertFalse(fired[i]);
+         assertNull(member[i]);
+         assertNull(memberId[i]);
+         assertFalse(isClient[i]);
+       }
+     }
+   }
+  
+   protected static int testClientMembershipEventsInClient_port;
+   private static int getTestClientMembershipEventsInClient_port() {
+     return testClientMembershipEventsInClient_port;
+   }
+   /**
+    * Tests notification of events in client process. Bridge clients detect
+    * server joins when the client connects to the server. If the server
+    * crashes or departs gracefully, the client will detect this as a crash.
+    */
+   public void testClientMembershipEventsInClient() throws Exception {
+     addExpectedException("IOException");
+     final boolean[] fired = new boolean[3];
+     final DistributedMember[] member = new DistributedMember[3];
+     final String[] memberId = new String[3];
+     final boolean[] isClient = new boolean[3];
+     
+     // create and register ClientMembershipListener in controller vm...
+     ClientMembershipListener listener = new ClientMembershipListener() {
+       public synchronized void memberJoined(ClientMembershipEvent event) {
+         getLogWriter().info("[testClientMembershipEventsInClient] memberJoined: " + event);
+         fired[JOINED] = true;
+         member[JOINED] = event.getMember();
+         memberId[JOINED] = event.getMemberId();
+         isClient[JOINED] = event.isClient();
+         notifyAll();
+       }
+       public synchronized void memberLeft(ClientMembershipEvent event) {
+         getLogWriter().info("[testClientMembershipEventsInClient] memberLeft: " + event);
+ //        fail("Please update testClientMembershipEventsInClient to handle memberLeft for BridgeServer.");
+       }
+       public synchronized void memberCrashed(ClientMembershipEvent event) {
+         getLogWriter().info("[testClientMembershipEventsInClient] memberCrashed: " + event);
+         fired[CRASHED] = true;
+         member[CRASHED] = event.getMember();
+         memberId[CRASHED] = event.getMemberId();
+         isClient[CRASHED] = event.isClient();
+         notifyAll();
+       }
+     };
+     ClientMembership.registerClientMembershipListener(listener);
+ 
+     final VM vm0 = Host.getHost(0).getVM(0);
+     final String name = this.getUniqueName();
+     final int[] ports = new int[1];
+ 
+     // create BridgeServer in vm0...
+     vm0.invoke(new CacheSerializableRunnable("Create BridgeServer") {
+       public void run2() throws CacheException {
+         try {
+           getLogWriter().info("[testClientMembershipEventsInClient] Create BridgeServer");
+           getSystem();
+           AttributesFactory factory = new AttributesFactory();
+           factory.setScope(Scope.LOCAL);
+           Region region = createRegion(name, factory.create());
+           assertNotNull(region);
+           assertNotNull(getRootRegion().getSubregion(name));
+           testClientMembershipEventsInClient_port = startBridgeServer(0);
+         }
+         catch(IOException e) {
+           getSystem().getLogWriter().fine(new Exception(e));
+           fail("Failed to start CacheServer on VM1: " + e.getMessage());
+         }
+       }
+     });
+     
+     // gather details for later creation of ConnectionPool...
+     ports[0] = vm0.invokeInt(ClientMembershipDUnitTest.class, 
+                              "getTestClientMembershipEventsInClient_port");
+     assertTrue(ports[0] != 0);
+ 
+     DistributedMember serverMember = (DistributedMember) vm0.invoke(ClientMembershipDUnitTest.class,
+     "getDistributedMember");
+ 
+     String serverMemberId = (String) vm0.invoke(ClientMembershipDUnitTest.class,
+                                                 "getMemberId");
+ 
+     getLogWriter().info("[testClientMembershipEventsInClient] ports[0]=" + ports[0]);
+     getLogWriter().info("[testClientMembershipEventsInClient] serverMember=" + serverMember);
+     getLogWriter().info("[testClientMembershipEventsInClient] serverMemberId=" + serverMemberId);
+ 
+     assertFalse(fired[JOINED]);
+     assertNull(member[JOINED]);
+     assertNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(member[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(member[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     
+     // sanity check...
+     getLogWriter().info("[testClientMembershipEventsInClient] sanity check");
+     DistributedMember test = new TestDistributedMember("test");
+     InternalClientMembership.notifyJoined(test, SERVER);
+     synchronized(listener) {
+       if (!fired[JOINED] && !fired[CRASHED]) {
+         listener.wait(2000);
+       }
+     }
+     
+     assertTrue(fired[JOINED]);
+     assertEquals(test, member[JOINED]);
+     assertEquals(test.getId(), memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(member[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(member[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+     
+     // create bridge client in controller vm...
+     getLogWriter().info("[testClientMembershipEventsInClient] create bridge client");
+     Properties config = new Properties();
+     config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+     config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+     getSystem(config);
+     
+     try {
+       getCache();
+       AttributesFactory factory = new AttributesFactory();
+       factory.setScope(Scope.LOCAL);
+       ClientServerTestCase.configureConnectionPool(factory, getServerHostName(Host.getHost(0)), ports, true, -1, -1, null);
+       createRegion(name, factory.create());
+       assertNotNull(getRootRegion().getSubregion(name));
+     }
+     catch (CacheException ex) {
+       fail("While creating Region on Edge", ex);
+     }
+     synchronized(listener) {
+       if (!fired[JOINED] && !fired[CRASHED]) {
+         listener.wait(60 * 1000);
+       }
+     }
+     
+     getLogWriter().info("[testClientMembershipEventsInClient] assert client detected server join");
+     
+     // first check the getCurrentServers() result
+     ClientCache clientCache = (ClientCache)getCache();
+     Set<InetSocketAddress> servers = clientCache.getCurrentServers();
+     assertTrue(!servers.isEmpty());
+     InetSocketAddress serverAddr = servers.iterator().next();
+     InetSocketAddress expectedAddr = new InetSocketAddress(serverMember.getHost(), ports[0]);
+     assertEquals(expectedAddr, serverAddr);
+     
+     // now check listener results
+     assertTrue(fired[JOINED]);
+     assertNotNull(member[JOINED]);
+     assertNotNull(memberId[JOINED]);
+     assertEquals(serverMember, member[JOINED]);
+     assertEquals(serverMemberId, memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(member[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(member[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     vm0.invoke(new SerializableRunnable("Stop BridgeServer") {
+       public void run() {
+         getLogWriter().info("[testClientMembershipEventsInClient] Stop BridgeServer");
+         stopBridgeServers(getCache());
+       }
+     });
+     synchronized(listener) {
+       if (!fired[JOINED] && !fired[CRASHED]) {
+         listener.wait(60 * 1000);
+       }
+     }
+     
+     getLogWriter().info("[testClientMembershipEventsInClient] assert client detected server departure");
+     assertFalse(fired[JOINED]);
+     assertNull(member[JOINED]);
+     assertNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(member[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertTrue(fired[CRASHED]);
+     assertNotNull(member[CRASHED]);
+     assertNotNull(memberId[CRASHED]);
+     assertEquals(serverMember, member[CRASHED]);
+     assertEquals(serverMemberId, memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+     
+     //now test that we redisover the bridge server
+     vm0.invoke(new CacheSerializableRunnable("Recreate BridgeServer") {
+       public void run2() throws CacheException {
+         try {
+           getLogWriter().info("[testClientMembershipEventsInClient] restarting BridgeServer");
+           startBridgeServer(ports[0]);
+         }
+         catch(IOException e) {
+           getSystem().getLogWriter().fine(new Exception(e));
+           fail("Failed to start CacheServer on VM1: " + e.getMessage());
+         }
+       }
+     });
+     synchronized(listener) {
+       if (!fired[JOINED] && !fired[CRASHED]) {
+         listener.wait(60 * 1000);
+       }
+     }
+     
+     getLogWriter().info("[testClientMembershipEventsInClient] assert client detected server recovery");
+     assertTrue(fired[JOINED]);
+     assertNotNull(member[JOINED]);
+     assertNotNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertEquals(serverMember, member[JOINED]);
+     assertEquals(serverMemberId, memberId[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(member[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(member[CRASHED]);
+     assertNull(memberId[CRASHED]);
+   }
+   
+   /**
+    * Tests notification of events in server process. Bridge servers detect
+    * client joins when the client connects to the server.
+    */
+   public void testClientMembershipEventsInServer() throws Exception {
+     final boolean[] fired = new boolean[3];
+     final DistributedMember[] member = new DistributedMember[3];
+     final String[] memberId = new String[3];
+     final boolean[] isClient = new boolean[3];
+     
+     // create and register ClientMembershipListener in controller vm...
+     ClientMembershipListener listener = new ClientMembershipListener() {
+       public synchronized void memberJoined(ClientMembershipEvent event) {
+         getLogWriter().info("[testClientMembershipEventsInServer] memberJoined: " + event);
+         fired[JOINED] = true;
+         member[JOINED] = event.getMember();
+         memberId[JOINED] = event.getMemberId();
+         isClient[JOINED] = event.isClient();
+         notifyAll();
+         assertFalse(fired[LEFT] || fired[CRASHED]);
+       }
+       public synchronized void memberLeft(ClientMembershipEvent event) {
+         getLogWriter().info("[testClientMembershipEventsInServer] memberLeft: " + event);
+         fired[LEFT] = true;
+         member[LEFT] = event.getMember();
+         memberId[LEFT] = event.getMemberId();
+         isClient[LEFT] = event.isClient();
+         notifyAll();
+         assertFalse(fired[JOINED] || fired[CRASHED]);
+       }
+       public synchronized void memberCrashed(ClientMembershipEvent event) {
+         getLogWriter().info("[testClientMembershipEventsInServer] memberCrashed: " + event);
+         fired[CRASHED] = true;
+         member[CRASHED] = event.getMember();
+         memberId[CRASHED] = event.getMemberId();
+         isClient[CRASHED] = event.isClient();
+         notifyAll();
+         assertFalse(fired[JOINED] || fired[LEFT]);
+       }
+     };
+     ClientMembership.registerClientMembershipListener(listener);
+ 
+     final VM vm0 = Host.getHost(0).getVM(0);
+     final String name = this.getUniqueName();
+     final int[] ports = new int[1];
+ 
+     // create BridgeServer in controller vm...
+     getLogWriter().info("[testClientMembershipEventsInServer] Create BridgeServer");
+     getSystem();
+     AttributesFactory factory = new AttributesFactory();
+     factory.setScope(Scope.LOCAL);
+     Region region = createRegion(name, factory.create());
+     assertNotNull(region);
+     assertNotNull(getRootRegion().getSubregion(name));
+     
+     ports[0] = startBridgeServer(0);
+     assertTrue(ports[0] != 0);
+     String serverMemberId = getMemberId();
+     DistributedMember serverMember = getDistributedMember();
+ 
+     getLogWriter().info("[testClientMembershipEventsInServer] ports[0]=" + ports[0]);
+     getLogWriter().info("[testClientMembershipEventsInServer] serverMemberId=" + serverMemberId);
+     getLogWriter().info("[testClientMembershipEventsInServer] serverMember=" + serverMember);
+ 
+     assertFalse(fired[JOINED]);
+     assertNull(member[JOINED]);
+     assertNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(member[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(member[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     
+     // sanity check...
+     getLogWriter().info("[testClientMembershipEventsInServer] sanity check");
+     DistributedMember test = new TestDistributedMember("test");
+     InternalClientMembership.notifyJoined(test, CLIENT);
+     synchronized(listener) {
+       if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+         listener.wait(2000);
+       }
+     }
+     assertTrue(fired[JOINED]);
+     assertEquals(test, member[JOINED]);
+     assertEquals(test.getId(), memberId[JOINED]);
+     assertTrue(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(member[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(member[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+     
+     final Host host = Host.getHost(0);
+     SerializableRunnable createConnectionPool =
+     new CacheSerializableRunnable("Create connectionPool") {
+       public void run2() throws CacheException {
+         getLogWriter().info("[testClientMembershipEventsInServer] create bridge client");
+         Properties config = new Properties();
+         config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+         config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+         getSystem(config);
+         AttributesFactory factory = new AttributesFactory();
+         factory.setScope(Scope.LOCAL);
+         ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, 2, null);
+         createRegion(name, factory.create());
+         assertNotNull(getRootRegion().getSubregion(name));
+       }
+     };
+ 
+     // create bridge client in vm0...
+     vm0.invoke(createConnectionPool);
+     String clientMemberId = (String) vm0.invoke(ClientMembershipDUnitTest.class,
+                                                 "getMemberId");
+     DistributedMember clientMember = (DistributedMember) vm0.invoke(ClientMembershipDUnitTest.class,
+                                                 "getDistributedMember");
+                                                 
+     synchronized(listener) {
+       if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+         listener.wait(60000);
+       }
+     }
+     
+     getLogWriter().info("[testClientMembershipEventsInServer] assert server detected client join");
+     assertTrue(fired[JOINED]);
+     assertEquals(member[JOINED] + " should equal " + clientMember,
+       clientMember, member[JOINED]);
+     assertEquals(memberId[JOINED] + " should equal " + clientMemberId,
+       clientMemberId, memberId[JOINED]);
+     assertTrue(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(member[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(member[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     pauseForClientToJoin();
+     
+     vm0.invoke(new SerializableRunnable("Stop bridge client") {
+       public void run() {
+         getLogWriter().info("[testClientMembershipEventsInServer] Stop bridge client");
+         getRootRegion().getSubregion(name).close();
+         Map m = PoolManager.getAll();
+         Iterator mit = m.values().iterator();
+         while(mit.hasNext()) {
+           Pool p = (Pool)mit.next();
+           p.destroy();
+         }
+       }
+     });
+ 
+     synchronized(listener) {
+       if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+         listener.wait(60000);
+       }
+     }
+     
+     getLogWriter().info("[testClientMembershipEventsInServer] assert server detected client left");
+     assertFalse(fired[JOINED]);
+     assertNull(member[JOINED]);
+     assertNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     assertTrue(fired[LEFT]);
+     assertEquals(clientMember, member[LEFT]);
+     assertEquals(clientMemberId, memberId[LEFT]);
+     assertTrue(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(member[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+ 
+     // reconnect bridge client to test for crashed event
+     vm0.invoke(createConnectionPool);
+     clientMemberId = (String) vm0.invoke(ClientMembershipDUnitTest.class,
+                                          "getMemberId");
+                                                 
+     synchronized(listener) {
+       if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+         listener.wait(60000);
+       }
+     }
+     
+     getLogWriter().info("[testClientMembershipEventsInServer] assert server detected client re-join");
+     assertTrue(fired[JOINED]);
+     assertEquals(clientMember, member[JOINED]);
+     assertEquals(clientMemberId, memberId[JOINED]);
+     assertTrue(isClient[JOINED]);
+     assertFalse(fired[LEFT]);
+     assertNull(member[LEFT]);
+     assertNull(memberId[LEFT]);
+     assertFalse(isClient[LEFT]);
+     assertFalse(fired[CRASHED]);
+     assertNull(member[CRASHED]);
+     assertNull(memberId[CRASHED]);
+     assertFalse(isClient[CRASHED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+     
+     pauseForClientToJoin();
+ 
+     ServerConnection.setForceClientCrashEvent(true);
+     try {
+       vm0.invoke(new SerializableRunnable("Stop bridge client") {
+         public void run() {
+           getLogWriter().info("[testClientMembershipEventsInServer] Stop bridge client");
+           getRootRegion().getSubregion(name).close();
+           Map m = PoolManager.getAll();
+           Iterator mit = m.values().iterator();
+           while(mit.hasNext()) {
+             Pool p = (Pool)mit.next();
+             p.destroy();
+           }
+         }
+       });
+   
+       synchronized(listener) {
+         if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+           listener.wait(60000);
+         }
+       }
+       
+       getLogWriter().info("[testClientMembershipEventsInServer] assert server detected client crashed");
+       assertFalse(fired[JOINED]);
+       assertNull(member[JOINED]);
+       assertNull(memberId[JOINED]);
+       assertFalse(isClient[JOINED]);
+       assertFalse(fired[LEFT]);
+       assertNull(member[LEFT]);
+       assertNull(memberId[LEFT]);
+       assertFalse(isClient[LEFT]);
+       assertTrue(fired[CRASHED]);
+       assertEquals(clientMember, member[CRASHED]);
+       assertEquals(clientMemberId, memberId[CRASHED]);
+       assertTrue(isClient[CRASHED]);
+     }
+     finally {
+       ServerConnection.setForceClientCrashEvent(false);
+     }
+   }
+   
+   /**
+    * The joined event fires when the first client handshake is processed.
+    * This pauses long enough to allow the rest of the client sockets to
+    * complete handshaking before making the client leave. Without doing this
+    * subsequent socket handshakes that are processed could fire join events
+    * after departure events and then a departure event again. If you see
+    * failures in testClientMembershipEventsInServer, try increasing this
+    * timeout.
+    */
+   private void pauseForClientToJoin() {
+     pause(2000);
+   }
+   
+   /** 
+    * Tests registration and event notification in conjunction with 
+    * disconnecting and reconnecting to DistributedSystem. 
+    */
+   public void testLifecycle() throws Exception {
+     final boolean[] fired = new boolean[3];
+     final DistributedMember[] member = new DistributedMember[3];
+     final String[] memberId = new String[3];
+     final boolean[] isClient = new boolean[3];
+     
+     // create and register ClientMembershipListener in controller vm...
+     ClientMembershipListener listener = new ClientMembershipListener() {
+       public synchronized void memberJoined(ClientMembershipEvent event) {
+         assertFalse(fired[JOINED]);
+         assertNull(member[JOINED]);
+         assertNull(memberId[JOINED]);
+         assertFalse(isClient[JOINED]);
+         fired[JOINED] = true;
+         member[JOINED] = event.getMember();
+         memberId[JOINED] = event.getMemberId();
+         isClient[JOINED] = event.isClient();
+         notifyAll();
+       }
+       public synchronized void memberLeft(ClientMembershipEvent event) {
+       }
+       public synchronized void memberCrashed(ClientMembershipEvent event) {
+       }
+     };
+     ClientMembership.registerClientMembershipListener(listener);
+     
+     // create loner in controller vm...
+     Properties config = new Properties();
+     config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+     config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+     getSystem(config);
+     
+     // assert that event is fired while connected
+     DistributedMember serverJoined = new TestDistributedMember("serverJoined");
+     InternalClientMembership.notifyJoined(serverJoined, SERVER);
+     synchronized(listener) {
+       if (!fired[JOINED]) {
+         listener.wait(2000);
+       }
+     }
+     assertTrue(fired[JOINED]);
+     assertEquals(serverJoined, member[JOINED]);
+     assertEquals(serverJoined.getId(), memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+     
+     // assert that event is NOT fired while disconnected
+     disconnectFromDS();
+     
+ 
+     InternalClientMembership.notifyJoined(serverJoined, SERVER);
+     synchronized(listener) {
+       listener.wait(20);
+     }
+     assertFalse(fired[JOINED]);
+     assertNull(member[JOINED]);
+     assertNull(memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+     resetArraysForTesting(fired, member, memberId, isClient);
+     
+     // assert that event is fired again after reconnecting
+     InternalDistributedSystem sys = getSystem(config);
+     assertTrue(sys.isConnected());
+ 
+     InternalClientMembership.notifyJoined(serverJoined, SERVER);
+     synchronized(listener) {
+       if (!fired[JOINED]) {
+         listener.wait(2000);
+       }
+     }
+     assertTrue(fired[JOINED]);
+     assertEquals(serverJoined, member[JOINED]);
+     assertEquals(serverJoined.getId(), memberId[JOINED]);
+     assertFalse(isClient[JOINED]);
+   }
+   
+   /**
+    * Starts up server in controller vm and 4 clients, then calls and tests
+    * ClientMembership.getConnectedClients(). 
+    */
+   public void testGetConnectedClients() throws Exception {
+     final String name = this.getUniqueName();
+     final int[] ports = new int[1];
++    
++    addExpectedException("ConnectException");
+ 
+     // create BridgeServer in controller vm...
+     getLogWriter().info("[testGetConnectedClients] Create BridgeServer");
+     getSystem();
+     AttributesFactory factory = new AttributesFactory();
+     factory.setScope(Scope.LOCAL);
+     Region region = createRegion(name, factory.create());
+     assertNotNull(region);
+     assertNotNull(getRootRegion().getSubregion(name));
+     
+     ports[0] = startBridgeServer(0);
+     assertTrue(ports[0] != 0);
+     String serverMemberId = getMemberId();
+ 
+     getLogWriter().info("[testGetConnectedClients] ports[0]=" + ports[0]);
+     getLogWriter().info("[testGetConnectedClients] serverMemberId=" + serverMemberId);
+ 
+     final Host host = Host.getHost(0);
+     SerializableRunnable createPool =
+     new CacheSerializableRunnable("Create connection pool") {
+       public void run2() throws CacheException {
+         getLogWriter().info("[testGetConnectedClients] create bridge client");
+         Properties config = new Properties();
+         config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+         config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+         getSystem(config);
+         AttributesFactory factory = new AttributesFactory();
+         factory.setScope(Scope.LOCAL);
+         ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, -1, null);
+         createRegion(name, factory.create());
+         assertNotNull(getRootRegion().getSubregion(name));
+       }
+     };
+ 
+     // create bridge client in vm0...
+     final String[] clientMemberIdArray = new String[host.getVMCount()];
+     
+     for (int i = 0; i < host.getVMCount(); i++) { 
+       final VM vm = Host.getHost(0).getVM(i);
+       vm.invoke(createPool);
+       clientMemberIdArray[i] =  String.valueOf(vm.invoke(
+         ClientMembershipDUnitTest.class, "getMemberId"));
+     }
+     Collection clientMemberIds = Arrays.asList(clientMemberIdArray);
+                                                 
+     {
+       final int expectedClientCount = clientMemberIds.size();
+       WaitCriterion wc = new WaitCriterion() {
+         public String description() {
+           return "wait for clients";
+         }
+         public boolean done() {
+           Map connectedClients = InternalClientMembership.getConnectedClients(false);
+           if (connectedClients == null) {
+             return false;
+           }
+           if (connectedClients.size() != expectedClientCount) {
+             return false;
+           }
+           return true;
+         }
+       };
+       waitForCriterion(wc, 10000, 100, false);
+     }
+     
+     Map connectedClients = InternalClientMembership.getConnectedClients(false);
+     assertNotNull(connectedClients);
+     assertEquals(clientMemberIds.size(), connectedClients.size());
+     for (Iterator iter = connectedClients.keySet().iterator(); iter.hasNext();) {
+       String connectedClient = (String)iter.next();
+       getLogWriter().info("[testGetConnectedClients] checking for client " + connectedClient);
+       assertTrue(clientMemberIds.contains(connectedClient));
+       getLogWriter().info("[testGetConnectedClients] count for connectedClient: " + 
+                           connectedClients.get(connectedClient));
+     }
+   }
+ 
+   /**
+    * Starts up 4 server and the controller vm as a client, then calls and tests
+    * ClientMembership.getConnectedServers(). 
+    */
+   public void testGetConnectedServers() throws Exception {
+     final Host host = Host.getHost(0);
+     final String name = this.getUniqueName();
+     final int[] ports = new int[host.getVMCount()];
+     
+     for (int i = 0; i < host.getVMCount(); i++) { 
+       final int whichVM = i;
+       final VM vm = Host.getHost(0).getVM(i);
+       vm.invoke(new CacheSerializableRunnable("Create bridge server") {
+         public void run2() throws CacheException {
+           // create BridgeServer in controller vm...
+           getLogWriter().info("[testGetConnectedServers] Create BridgeServer");
+           getSystem();
+           AttributesFactory factory = new AttributesFactory();
+           factory.setScope(Scope.LOCAL);
+           Region region = createRegion(name+"_"+whichVM, factory.create());
+           assertNotNull(region);
+           assertNotNull(getRootRegion().getSubregion(name+"_"+whichVM));
+           region.put("KEY-1", "VAL-1");
+           
+           try {
+             testGetConnectedServers_port = startBridgeServer(0);
+           }
+           catch (IOException e) {
+             getLogWriter().error("startBridgeServer threw IOException", e);
+             fail("startBridgeServer threw IOException " + e.getMessage());
+           }
+           
+           assertTrue(testGetConnectedServers_port != 0);
+       
+           getLogWriter().info("[testGetConnectedServers] port=" + 
+             ports[whichVM]);
+           getLogWriter().info("[testGetConnectedServers] serverMemberId=" + 
+             getDistributedMember());
+         }
+       });
+       ports[whichVM] = vm.invokeInt(ClientMembershipDUnitTest.class, 
+                                     "getTestGetConnectedServers_port");
+       assertTrue(ports[whichVM] != 0);
+     }
+     
+     getLogWriter().info("[testGetConnectedServers] create bridge client");
+     Properties config = new Properties();
+     config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+     config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+     getSystem(config);
+     getCache();
+     
+     AttributesFactory factory = new AttributesFactory();
+     factory.setScope(Scope.LOCAL);
+ 
+     for (int i = 0; i < ports.length; i++) {
+       getLogWriter().info("[testGetConnectedServers] creating connectionpool for " + 
+         getServerHostName(host) + " " + ports[i]);
+       int[] thisServerPorts = new int[] { ports[i] };
+       ClientServerTestCase.configureConnectionPoolWithName(factory, getServerHostName(host), thisServerPorts, false, -1, -1, null,"pooly"+i);
+       Region region = createRegion(name+"_"+i, factory.create());
+       assertNotNull(getRootRegion().getSubregion(name+"_"+i));
+       region.get("KEY-1");
+     }
+ 
+     {
+       final int expectedVMCount = host.getVMCount();
+       WaitCriterion wc = new WaitCriterion() {
+         public String description() {
+           return "wait for pools and servers";
+         }
+         public boolean done() {
+           if (PoolManager.getAll().size() != expectedVMCount) {
+             return false;
+           }
+           Map connectedServers = InternalClientMembership.getConnectedServers();
+           if (connectedServers == null) {
+             return false;
+           }
+           if (connectedServers.size() != expectedVMCount) {
+             return false;
+           }
+           return true;
+         }
+       };
+       waitForCriterion(wc, 10000, 100, false);
+     }
+ 
+     {
+       assertEquals(host.getVMCount(), PoolManager.getAll().size());
+       
+     }
+     
+     Map connectedServers = InternalClientMembership.getConnectedServers();
+     assertNotNull(connectedServers);
+     assertEquals(host.getVMCount(), connectedServers.size());
+     for (Iterator iter = connectedServers.keySet().iterator(); iter.hasNext();) {
+       String connectedServer = (String) iter.next();
+       getLogWriter().info("[testGetConnectedServers]  value for connectedServer: " + 
+                           connectedServers.get(connectedServer));
+     }
+   }
+ 
+   protected static int testGetConnectedServers_port;
+   private static int getTestGetConnectedServers_port() {
+     return testGetConnectedServers_port;
+   }
+ 
+   /**
+    * Tests getConnectedClients(boolean onlyClientsNotifiedByThisServer) where
+    * onlyClientsNotifiedByThisServer is true.
+    */
+   public void testGetNotifiedClients() throws Exception {
+     final Host host = Host.getHost(0);
+     final String name = this.getUniqueName();
+     final int[] ports = new int[host.getVMCount()];
+     
+     for (int i = 0; i < host.getVMCount(); i++) { 
+       final int whichVM = i;
+       final VM vm = Host.getHost(0).getVM(i);
+       vm.invoke(new CacheSerializableRunnable("Create bridge server") {
+         public void run2() throws CacheException {
+           // create BridgeServer in controller vm...
+           getLogWriter().info("[testGetNotifiedClients] Create BridgeServer");
+           getSystem();
+           AttributesFactory factory = new AttributesFactory();
+           Region region = createRegion(name, factory.create());
+           assertNotNull(region);
+           assertNotNull(getRootRegion().getSubregion(name));
+           region.put("KEY-1", "VAL-1");
+           
+           try {
+             testGetNotifiedClients_port = startBridgeServer(0);
+           }
+           catch (IOException e) {
+             getLogWriter().error("startBridgeServer threw IOException", e);
+             fail("startBridgeServer threw IOException " + e.getMessage());
+           }
+           
+           assertTrue(testGetNotifiedClients_port != 0);
+       
+           getLogWriter().info("[testGetNotifiedClients] port=" + 
+             ports[whichVM]);
+           getLogWriter().info("[testGetNotifiedClients] serverMemberId=" + 
+             getMemberId());
+         }
+       });
+       ports[whichVM] = vm.invokeInt(ClientMembershipDUnitTest.class, 
+                                     "getTestGetNotifiedClients_port");
+       assertTrue(ports[whichVM] != 0);
+     }
+     
+     getLogWriter().info("[testGetNotifiedClients] create bridge client");
+     Properties config = new Properties();
+     config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+     config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+     getSystem(config);
+     getCache();
+     
+     AttributesFactory factory = new AttributesFactory();
+     factory.setScope(Scope.LOCAL);
+ 
+     getLogWriter().info("[testGetNotifiedClients] creating connection pool");
+     ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, -1, null);
+     Region region = createRegion(name, factory.create());
+     assertNotNull(getRootRegion().getSubregion(name));
+     region.registerInterest("KEY-1");
+     region.get("KEY-1");
+ 
+     final String clientMemberId = getMemberId();
+     
+     pauseForClientToJoin();
+     
+     // assertions go here
+     int[] clientCounts = new int[host.getVMCount()];
+     
+     // only one server vm will have that client for updating
+     for (int i = 0; i < host.getVMCount(); i++) { 
+       final int whichVM = i;
+       final VM vm = Host.getHost(0).getVM(i);
+       vm.invoke(new CacheSerializableRunnable("Create bridge server") {
+         public void run2() throws CacheException {
+           Map clients = InternalClientMembership.getConnectedClients(true);
+           assertNotNull(clients);
+           testGetNotifiedClients_clientCount = clients.size();
+           if (testGetNotifiedClients_clientCount > 0) {
+             // assert that the clientMemberId matches
+             assertEquals(clientMemberId, clients.keySet().iterator().next());
+           }
+         }
+       });
+       clientCounts[whichVM] = vm.invokeInt(ClientMembershipDUnitTest.class, 
+                               "getTestGetNotifiedClients_clientCount");
+     }
+     
+     // only one server should have a notifier for this client...
+     int totalClientCounts = 0;
+     for (int i = 0; i < clientCounts.length; i++) {
+       totalClientCounts += clientCounts[i];
+     }
+     // this assertion fails because the count is 4
+     //assertEquals(1, totalClientCounts);
+   }
+   protected static int testGetNotifiedClients_port;
+   private static int getTestGetNotifiedClients_port() {
+     return testGetNotifiedClients_port;
+   }
+   protected static int testGetNotifiedClients_clientCount;
+   private static int getTestGetNotifiedClients_clientCount() {
+     return testGetNotifiedClients_clientCount;
+   }
+ 
+   // Simple DistributedMember implementation
+   static final class TestDistributedMember implements DistributedMember {
+     
+     private String host;
+     
+     public TestDistributedMember(String host) {
+       this.host = host;
+     }
+ 
+     public String getName() {
+       return "";
+     }
+ 
+     public String getHost() {
+       return this.host;
+     }
+ 
+     public Set getRoles() {
+       return new HashSet();
+     }
+ 
+     public int getProcessId() {
+       return 0;
+     }
+ 
+     public String getId() {
+       return this.host;
+     }
+     
+     public int compareTo(DistributedMember o) {
+       if ((o == null) || !(o instanceof TestDistributedMember)) {
+         throw new InternalGemFireException("Invalidly comparing TestDistributedMember to " + o);
+       }
+       
+       TestDistributedMember tds = (TestDistributedMember) o;
+       return getHost().compareTo(tds.getHost());
+     }
+     
+     public boolean equals(Object obj) {
+       if ((obj == null) || !(obj instanceof TestDistributedMember)) {
+         return false;
+       }
+       return compareTo((TestDistributedMember)obj) == 0;
+     }
+     
+     public int hashCode() {
+       return getHost().hashCode();
+     }
+     
+     public DurableClientAttributes getDurableClientAttributes() {
+       
+       return null;
+     }
+ 
+     public List<String> getGroups() {
+       return Collections.emptyList();
+     }
+   }
+ }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerTestCase.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerTestCase.java
index 0000000,c645549..8e2f482
mode 000000,100644..100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientServerTestCase.java
@@@ -1,0 -1,376 +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";
++        return "bridge never finished connecting: " + system.getMemberId();
+       }
+ 
+       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 waitMillis = 10000;
+     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/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/ProductUseLogDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
index 1654f86,7379c86..641a267
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
@@@ -81,12 -81,14 +81,13 @@@ public class FDDUnitTest extends CacheT
        }
      }
      try {
+       StringBuffer incaseOfFailure = new StringBuffer();
        final int[] port = AvailablePortHelper.getRandomAvailableTCPPorts(3);
-       int numThreads = 10;
 -      final int mcastPort = AvailablePortHelper.getRandomAvailableUDPPort();
+       int numThreads = 30;
  
 -      startCacheServer(vm0, port[0], mcastPort);
 -      startCacheServer(vm1, port[1], mcastPort);
 -      startCacheServer(vm2, port[2], mcastPort);
 +      startCacheServer(vm0, port[0]);
 +      startCacheServer(vm1, port[1]);
 +      startCacheServer(vm2, port[2]);
  
        createRegion(vm0, "portfolios", RegionShortcut.PARTITION_REDUNDANT);
        createRegion(vm1, "portfolios", RegionShortcut.PARTITION_REDUNDANT);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug41957DUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientMessagesRegionCreationAndDestroyJUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerGetAllDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentMapOpsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/Bug51193DUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug48571DUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/Bug43684DUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientReconnectDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableClientStatsDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/DurableRegistrationDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/InterestListFailoverDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/java/dunit/DistributedTestCase.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/dunit/DistributedTestCase.java
index 30d0f12,d9f88ca..1acd0c2
--- a/gemfire-core/src/test/java/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
@@@ -756,9 -799,11 +756,9 @@@ public abstract class DistributedTestCa
      LogWrapper.close();
      ClientProxyMembershipID.system = null;
      MultiVMRegionTestCase.CCRegion = null;
-     InternalBridgeMembership.unregisterAllListeners();
+     InternalClientMembership.unregisterAllListeners();
      ClientStatsManager.cleanupForTests();
      unregisterInstantiatorsInThisVM();
 -    GemFireTracer.DEBUG = Boolean.getBoolean("DistributionManager.DEBUG_JAVAGROUPS");
 -    Protocol.trace = GemFireTracer.DEBUG;
      DistributionMessageObserver.setInstance(null);
      QueryObserverHolder.reset();
      DiskStoreObserver.setInstance(null);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------



[33/38] incubator-geode git commit: GEODE-189: Configure Log4J 2 using classpath instead of system property

Posted by bs...@apache.org.
GEODE-189: Configure Log4J 2 using classpath instead of system property

Rename log4j2-default.xml to log4j2.xml. Move log4j2.xml and
log4j2-cli.xml to the root of the gemfire-core JAR file. By default
Log4J 2 will find and use the log4j2.xml in the JAR file unless the
user places a custom config file earlier in the classpath or specifies
the log4j.configurationFile system property.

The new JGroups JAR also contains log4j2.xml in its root, so we need to
ensure that gemfire-core JAR is on the classpath before JGroups.

Upgrade System-Rules JUnit library to 1.12.1.


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

Branch: refs/heads/feature/GEODE-77
Commit: 21743ec131735cb7c3fd11c6a4ce3e18b79b06f9
Parents: 8d1ada3
Author: Kirk Lund <kl...@pivotal.io>
Authored: Thu Aug 13 11:36:43 2015 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Wed Oct 14 10:28:46 2015 -0700

----------------------------------------------------------------------
 build.gradle                                    |   2 +-
 gemfire-assembly/build.gradle                   |   9 +-
 gemfire-assembly/src/main/dist/bin/gfsh         |   2 +-
 gemfire-assembly/src/main/dist/bin/gfsh.bat     |   2 +-
 .../gemfire/internal/logging/LogService.java    | 128 ++++++++-----------
 .../internal/logging/log4j/Configurator.java    |   5 +
 .../internal/logging/log4j/log4j2-cli.xml       |  17 ---
 .../internal/logging/log4j/log4j2-default.xml   |  21 ---
 gemfire-core/src/main/resources/log4j2-cli.xml  |  17 +++
 gemfire-core/src/main/resources/log4j2.xml      |  22 ++++
 .../logging/LogServiceIntegrationJUnitTest.java |  98 +++++++-------
 .../internal/logging/LogServiceJUnitTest.java   |  58 +++++----
 .../LogServiceUserDirIntegrationJUnitTest.java  |  70 ----------
 .../logging/LoggingIntegrationTestSuite.java    |   6 +-
 .../internal/logging/LoggingUnitTestSuite.java  |  15 ++-
 .../logging/MergeLogFilesJUnitTest.java         |   4 +-
 .../log4j/FastLoggerIntegrationJUnitTest.java   |   4 +-
 .../FastLoggerWithDefaultConfigJUnitTest.java   |   4 +-
 18 files changed, 208 insertions(+), 276 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index c1534de..f2aa561 100755
--- a/build.gradle
+++ b/build.gradle
@@ -284,7 +284,7 @@ subprojects {
     compile 'org.springframework:spring-web:3.2.12.RELEASE'
     compile 'org.springframework:spring-webmvc:3.2.12.RELEASE'
 
-    testCompile 'com.github.stefanbirkner:system-rules:1.9.0'
+    testCompile 'com.github.stefanbirkner:system-rules:1.12.1'
     testCompile 'edu.umd.cs.mtc:multithreadedtc:1.01'
     testCompile 'junit:junit:4.12'
     testCompile 'org.assertj:assertj-core:2.1.0'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index f65930d..b351729 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -154,12 +154,9 @@ distributions {
       into ('config') {
         from defaultCacheConfig
         from defaultDistributionConfig
-
-        from (project(':gemfire-core').sourceSets.main.resources.files.find { 
-          it.name == 'log4j2-default.xml' 
-        }) {
-          rename 'log4j2-default.xml', 'log4j2.xml'
-        }
+        from (project(':gemfire-core').sourceSets.main.resources.files.find {
+          it.name == 'log4j2.xml' 
+        })
       }
       
       into ('lib') {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-assembly/src/main/dist/bin/gfsh
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/bin/gfsh b/gemfire-assembly/src/main/dist/bin/gfsh
index e700aae..bcb5a3d 100755
--- a/gemfire-assembly/src/main/dist/bin/gfsh
+++ b/gemfire-assembly/src/main/dist/bin/gfsh
@@ -134,5 +134,5 @@ fi
 # This enables job control and monitoring, effectively switching on the trap
 # handler to run when the java exe exits, regardless of how (even if killed).
 set -bm
-"$GF_JAVA" -Dgfsh=true -Dlog4j.configurationFile=/com/gemstone/gemfire/internal/logging/log4j/log4j2-cli.xml ${JLINE_TERMINAL} -classpath "${CLASSPATH}" $JAVA_ARGS $LAUNCHER  "$@"
+"$GF_JAVA" -Dgfsh=true -Dlog4j.configurationFile=classpath:log4j2-cli.xml ${JLINE_TERMINAL} -classpath "${CLASSPATH}" $JAVA_ARGS $LAUNCHER  "$@"
 exit $?

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-assembly/src/main/dist/bin/gfsh.bat
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/main/dist/bin/gfsh.bat b/gemfire-assembly/src/main/dist/bin/gfsh.bat
index c4a9480..10c2726 100755
--- a/gemfire-assembly/src/main/dist/bin/gfsh.bat
+++ b/gemfire-assembly/src/main/dist/bin/gfsh.bat
@@ -69,5 +69,5 @@ REM  Consider java is from JDK
 @if defined JAVA_ARGS (
 @set JAVA_ARGS="%JAVA_ARGS%"
 )
-@"%GF_JAVA%" -Dgfsh=true -Dlog4j.configurationFile=/com/gemstone/gemfire/internal/logging/log4j/log4j2-cli.xml %JAVA_ARGS% %LAUNCHER% %*
+@"%GF_JAVA%" -Dgfsh=true -Dlog4j.configurationFile=classpath:log4j2-cli.xml %JAVA_ARGS% %LAUNCHER% %*
 :done

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
index 637f8ca..a24343c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
@@ -9,13 +9,28 @@ package com.gemstone.gemfire.internal.logging;
 
 import java.beans.PropertyChangeEvent;
 import java.beans.PropertyChangeListener;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
 
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Source;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import javax.xml.validation.Validator;
+
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -25,12 +40,27 @@ import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.config.AppenderRef;
 import org.apache.logging.log4j.core.config.Configuration;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.apache.logging.log4j.core.config.ConfigurationSource;
+import org.apache.logging.log4j.core.config.FileConfigurationMonitor;
 import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.apache.logging.log4j.core.config.Node;
+import org.apache.logging.log4j.core.config.status.StatusConfiguration;
+import org.apache.logging.log4j.core.config.xml.XmlConfiguration;
 import org.apache.logging.log4j.core.filter.AbstractFilterable;
 import org.apache.logging.log4j.core.lookup.Interpolator;
+import org.apache.logging.log4j.core.lookup.StrLookup;
 import org.apache.logging.log4j.core.lookup.StrSubstitutor;
+import org.apache.logging.log4j.core.util.Closer;
+import org.apache.logging.log4j.core.util.Loader;
+import org.apache.logging.log4j.core.util.Patterns;
 import org.apache.logging.log4j.status.StatusLogger;
 import org.apache.logging.log4j.util.PropertiesUtil;
+import org.w3c.dom.Attr;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
 
 import com.gemstone.gemfire.InternalGemFireError;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -66,20 +96,21 @@ public class LogService extends LogManager {
 
   private static final PropertyChangeListener propertyChangeListener = new PropertyChangeListenerImpl();
   
-  public static final String DEFAULT_CONFIG = "/com/gemstone/gemfire/internal/logging/log4j/log4j2-default.xml";
-  public static final String CLI_CONFIG = "/com/gemstone/gemfire/internal/logging/log4j/log4j2-cli.xml";
+  public static final String DEFAULT_CONFIG = "/log4j2.xml";
+  public static final String CLI_CONFIG = "/log4j2-cli.xml";
 
+  /**
+   * Name of variable that is set to "true" in log4j2.xml to indicate that it is the default gemfire config xml.
+   */
+  private static final String GEMFIRE_DEFAULT_PROPERTY = "gemfire-default";
+  
   /** Protected by static synchronization. Used for removal and adding stdout back in. */
   private static Appender stdoutAppender;
   
-  /** Set with info for Banner to log anytime LogService is reinitialized */
-  private static String configFileInformation;
-  
   static {
     init();
   }
   private static void init() {
-    setLog4jConfigFileProperty();
     LoggerContext context = ((org.apache.logging.log4j.core.Logger) LogManager.getLogger(BASE_LOGGER_NAME, GemFireParameterizedMessageFactory.INSTANCE)).getContext();
     context.removePropertyChangeListener(propertyChangeListener);
     context.addPropertyChangeListener(propertyChangeListener);
@@ -111,83 +142,19 @@ public class LogService extends LogManager {
   }
   
   public static boolean isUsingGemFireDefaultConfig() {
-    final String configFileName = new StrSubstitutor(new Interpolator()).replace(
-        PropertiesUtil.getProperties().getStringProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY));
-    return configFileName == null ? false : configFileName.contains(DEFAULT_CONFIG);
-  }
-  
-  /**
-   * Check to see if the user has specified a Log4j configuration file.  If not, attempt
-   * to find a GemFire Log4j configuration file in various locations.
-   * 
-   * @return true if log4j.configurationFile property was set; false if it was unchanged
-   */
-  private static final boolean setLog4jConfigFileProperty() {
-    // fix bug #52175
-    final URL configInClasspath = ConfigLocator.findConfigInClasspath();
-    if (configInClasspath != null ) {
-      // Log4J 2 will find the configuration file in classpath so do nothing
-      configFileInformation = "Using log4j configuration found in classpath: '" + configInClasspath.toString() + "'";
-      StatusLogger.getLogger().info(configFileInformation);
-      return false;
-    }
+    final Configuration config = ((org.apache.logging.log4j.core.Logger)
+        LogManager.getLogger(ROOT_LOGGER_NAME, GemFireParameterizedMessageFactory.INSTANCE)).getContext().getConfiguration();
     
-    // If the user set the log4j system property then there's nothing else to do.
-    final String configFileName = new StrSubstitutor(new Interpolator()).replace(
-        PropertiesUtil.getProperties().getStringProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY));
-    if (configFileName != null) {
-      final URL configUrl = LogService.class.getResource(configFileName);// log4j2-cli.xml is non-null, external is null
-      if (configUrl == null) {
-        //We will let log4j2 handle the null case and just log what file we are attempting to use
-        configFileInformation = "Using log4j configuration file specified by " + ConfigurationFactory.CONFIGURATION_FILE_PROPERTY + ": '" + configFileName + "'";
-        StatusLogger.getLogger().info(configFileInformation);
-        return false;
-      }
-      else {
-        //If the resource can be found and in cases where the resource is in gemfire jar,
-        //we set the log location to the file that was found
-
-        // must change from Java resource syntax to Java URL syntax (GEODE-232)
-        // jar:file:/export/latvia1/users/klund/dev/asf-geode/gemfire-assembly/build/install/apache-geode/lib/gemfire-core-1.0.0-incubating-SNAPSHOT.jar!/com/gemstone/gemfire/internal/logging/log4j/log4j2-cli.xml
-        
-        String configFilePropertyValue = configUrl.toString();
-        // configFileName is Java resource syntax, configFilePropertyValue is URL syntax as required by log4j2
-        
-        System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, configFilePropertyValue);
-        configFileInformation = "Using log4j configuration file specified by " + ConfigurationFactory.CONFIGURATION_FILE_PROPERTY + ": '" + configFilePropertyValue + "'";
-        StatusLogger.getLogger().info(configFileInformation);
-        return true;
-      }
-    }
+    final StrSubstitutor sub = config.getStrSubstitutor();
+    final StrLookup resolver = sub.getVariableResolver();
     
-//    // set log4j.configurationFactory to be our optimized version
-//    final String factory = GemFireXmlConfigurationFactory.class.getName();
-//    System.setProperty(ConfigurationFactory.CONFIGURATION_FACTORY_PROPERTY, factory);
-//    StatusLogger.getLogger().debug("Using log4j configuration factory '{}'", factory);
+    final String value = resolver.lookup(GEMFIRE_DEFAULT_PROPERTY);
     
-    // If one of the default log4j config files exists in the current directory then use it.
-    File log4jConfigFile = findLog4jConfigInCurrentDir();
-    if (log4jConfigFile != null) {
-      String filePath = IOUtils.tryGetCanonicalPathElseGetAbsolutePath(log4jConfigFile);
-      String value = new File(filePath).toURI().toString();
-      String configFilePropertyValue = new File(filePath).toURI().toString();
-      System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, configFilePropertyValue);
-      configFileInformation = "Setting " + ConfigurationFactory.CONFIGURATION_FILE_PROPERTY + " to specify log4j configuration file in current directory: '" + configFilePropertyValue + "'";
-      StatusLogger.getLogger().debug(configFileInformation);
-      return true;
-    }
-
-    // Use the log4j config file found on the classpath in the gemfire jar file.
-    final URL configUrl = LogService.class.getResource(DEFAULT_CONFIG);
-    String configFilePropertyValue = configUrl.toString();
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, configFilePropertyValue);
-    configFileInformation = "Setting " + ConfigurationFactory.CONFIGURATION_FILE_PROPERTY + " to specify log4j configuration file: '" + configFilePropertyValue + "'";
-    StatusLogger.getLogger().info(configFileInformation);
-    return true;
+    return "true".equals(value);
   }
   
   public static String getConfigInformation() {
-    return configFileInformation;
+    return getConfiguration().getConfigurationSource().toString();
   }
 
   /**
@@ -255,6 +222,12 @@ public class LogService extends LogManager {
     return new Throwable().getStackTrace()[depth].getClassName();
   }
 
+  public static Configuration getConfiguration() {
+    final Configuration config = ((org.apache.logging.log4j.core.Logger)
+        LogManager.getLogger(ROOT_LOGGER_NAME, GemFireParameterizedMessageFactory.INSTANCE)).getContext().getConfiguration();
+    return config;
+  }
+  
   public static void configureFastLoggerDelegating() {
     final Configuration config = ((org.apache.logging.log4j.core.Logger)
         LogManager.getLogger(ROOT_LOGGER_NAME, GemFireParameterizedMessageFactory.INSTANCE)).getContext().getConfiguration();
@@ -300,7 +273,6 @@ public class LogService extends LogManager {
   }
   
   public static LoggerConfig getRootLoggerConfig() {
-    //return ((org.apache.logging.log4j.core.Logger)LogService.getLogger()).getContext().getConfiguration().getLoggerConfig(LogManager.getRootLogger().getName());
     return Configurator.getLoggerConfig(LogManager.getRootLogger().getName());
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java
index e07f214..f1509df 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java
@@ -10,6 +10,7 @@ import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.config.AppenderRef;
 import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.config.ConfigurationSource;
 import org.apache.logging.log4j.core.config.LoggerConfig;
 import org.apache.logging.log4j.core.config.Property;
 import org.apache.logging.log4j.core.filter.AbstractFilterable;
@@ -125,6 +126,10 @@ public class Configurator {
     return config.hasFilter();
   }
   
+  public static String getConfigurationSourceLocation(final Configuration config) {
+    return config.getConfigurationSource().getLocation();
+  }
+  
   public static boolean hasAppenderFilter(final Configuration config) {
     for (Appender appender : config.getAppenders().values()) {
       if (appender instanceof AbstractFilterable) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/logging/log4j/log4j2-cli.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/logging/log4j/log4j2-cli.xml b/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/logging/log4j/log4j2-cli.xml
deleted file mode 100644
index 14b7075..0000000
--- a/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/logging/log4j/log4j2-cli.xml
+++ /dev/null
@@ -1,17 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<Configuration status="FATAL" shutdownHook="disable" packages="com.gemstone.gemfire.internal.logging.log4j">
-  <Properties>
-    <Property name="gemfire-pattern">[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} &lt;%thread&gt; tid=%tid] %message%n%throwable%n</Property>
-  </Properties>
-  <Appenders>
-    <Console name="STDOUT" target="SYSTEM_OUT">
-      <PatternLayout pattern="${gemfire-pattern}"/>
-    </Console>
-  </Appenders>
-  <Loggers>
-    <Logger name="com.gemstone" level="WARN" additivity="true"/>
-    <Root level="WARN">
-      <AppenderRef ref="STDOUT"/>
-    </Root>
-  </Loggers>
-</Configuration>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/logging/log4j/log4j2-default.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/logging/log4j/log4j2-default.xml b/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/logging/log4j/log4j2-default.xml
deleted file mode 100644
index a6e8cd8..0000000
--- a/gemfire-core/src/main/resources/com/gemstone/gemfire/internal/logging/log4j/log4j2-default.xml
+++ /dev/null
@@ -1,21 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<Configuration status="FATAL" shutdownHook="disable" packages="com.gemstone.gemfire.internal.logging.log4j">
-  <Properties>
-    <Property name="gemfire-pattern">[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} &lt;%thread&gt; tid=%tid] %message%n%throwable%n</Property>
-  </Properties>
-  <Appenders>
-    <Console name="STDOUT" target="SYSTEM_OUT">
-      <PatternLayout pattern="${gemfire-pattern}"/>
-    </Console>
-  </Appenders>
-  <Loggers>
-    <Logger name="com.gemstone" level="INFO" additivity="true">
-    <filters>
-      <MarkerFilter marker="GEMFIRE_VERBOSE" onMatch="DENY" onMismatch="NEUTRAL"/>
-    </filters>
-    </Logger>
-    <Root level="INFO">
-      <AppenderRef ref="STDOUT"/>
-    </Root>
-  </Loggers>
-</Configuration>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/main/resources/log4j2-cli.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/resources/log4j2-cli.xml b/gemfire-core/src/main/resources/log4j2-cli.xml
new file mode 100644
index 0000000..14b7075
--- /dev/null
+++ b/gemfire-core/src/main/resources/log4j2-cli.xml
@@ -0,0 +1,17 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Configuration status="FATAL" shutdownHook="disable" packages="com.gemstone.gemfire.internal.logging.log4j">
+  <Properties>
+    <Property name="gemfire-pattern">[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} &lt;%thread&gt; tid=%tid] %message%n%throwable%n</Property>
+  </Properties>
+  <Appenders>
+    <Console name="STDOUT" target="SYSTEM_OUT">
+      <PatternLayout pattern="${gemfire-pattern}"/>
+    </Console>
+  </Appenders>
+  <Loggers>
+    <Logger name="com.gemstone" level="WARN" additivity="true"/>
+    <Root level="WARN">
+      <AppenderRef ref="STDOUT"/>
+    </Root>
+  </Loggers>
+</Configuration>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/main/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/resources/log4j2.xml b/gemfire-core/src/main/resources/log4j2.xml
new file mode 100755
index 0000000..a21a333
--- /dev/null
+++ b/gemfire-core/src/main/resources/log4j2.xml
@@ -0,0 +1,22 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Configuration status="FATAL" shutdownHook="disable" packages="com.gemstone.gemfire.internal.logging.log4j">
+  <Properties>
+    <Property name="gemfire-pattern">[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} &lt;%thread&gt; tid=%tid] %message%n%throwable%n</Property>
+    <Property name="gemfire-default">true</Property>
+  </Properties>
+  <Appenders>
+    <Console name="STDOUT" target="SYSTEM_OUT">
+      <PatternLayout pattern="${gemfire-pattern}"/>
+    </Console>
+  </Appenders>
+  <Loggers>
+    <Logger name="com.gemstone" level="INFO" additivity="true">
+    <filters>
+      <MarkerFilter marker="GEMFIRE_VERBOSE" onMatch="DENY" onMismatch="NEUTRAL"/>
+    </filters>
+    </Logger>
+    <Root level="INFO">
+      <AppenderRef ref="STDOUT"/>
+    </Root>
+  </Loggers>
+</Configuration>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceIntegrationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceIntegrationJUnitTest.java
index 6425c1f..fb54061 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceIntegrationJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceIntegrationJUnitTest.java
@@ -7,19 +7,20 @@ import java.io.File;
 import java.net.URL;
 
 import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.Logger;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
 import org.apache.logging.log4j.status.StatusLogger;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.contrib.java.lang.system.SystemErrRule;
+import org.junit.contrib.java.lang.system.SystemOutRule;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.ExternalResource;
 import org.junit.rules.TemporaryFolder;
 
-import com.gemstone.gemfire.internal.ClassPathLoader;
 import com.gemstone.gemfire.internal.logging.log4j.Configurator;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
@@ -31,8 +32,14 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 @Category(IntegrationTest.class)
 public class LogServiceIntegrationJUnitTest {
   
-  private String beforeConfigFileProp;
-  private Level beforeLevel;
+  private static final String DEFAULT_CONFIG_FILE_NAME = "log4j2.xml";
+  private static final String CLI_CONFIG_FILE_NAME = "log4j2-cli.xml";
+  
+  @Rule
+  public final SystemErrRule systemErrRule = new SystemErrRule().enableLog();
+  
+  @Rule
+  public final SystemOutRule systemOutRule = new SystemOutRule().enableLog();
   
   @Rule
   public final TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -53,6 +60,7 @@ public class LogServiceIntegrationJUnitTest {
     protected void after() {
       Configurator.shutdown();
       
+      System.clearProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
       if (beforeConfigFileProp != null) {
         System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, beforeConfigFileProp);
       }
@@ -63,6 +71,9 @@ public class LogServiceIntegrationJUnitTest {
     }
   };
   
+  private String beforeConfigFileProp;
+  private Level beforeLevel;
+  
   private URL defaultConfigUrl;
   private URL cliConfigUrl;
   
@@ -72,9 +83,16 @@ public class LogServiceIntegrationJUnitTest {
     this.cliConfigUrl = LogService.class.getResource(LogService.CLI_CONFIG);
   }
   
+  @After
+  public void after() {
+    // if either of these fail then log4j2 probably logged a failure to stdout
+    assertThat(this.systemErrRule.getLog()).isEmpty();
+    assertThat(this.systemOutRule.getLog()).isEmpty();
+  }
+  
   @Test
-  public void shouldPreferConfigInConfigurationFilePropertyIfSet() throws Exception {
-    final File configFile = this.temporaryFolder.newFile("log4j2.xml");
+  public void shouldPreferConfigurationFilePropertyIfSet() throws Exception {
+    final File configFile = this.temporaryFolder.newFile(DEFAULT_CONFIG_FILE_NAME);
     final String configFileName = configFile.toURI().toString();
     System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, configFileName);
     writeConfigFile(configFile, Level.DEBUG);
@@ -91,21 +109,7 @@ public class LogServiceIntegrationJUnitTest {
     LogService.reconfigure();
     
     assertThat(LogService.isUsingGemFireDefaultConfig()).as(LogService.getConfigInformation()).isTrue();
-    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).isEqualTo(this.defaultConfigUrl.toString());
-  }
-  
-  @Test
-  public void defaultConfigShouldBeLoadableAsResource() {
-    final URL configUrlFromLogService = LogService.class.getResource(LogService.DEFAULT_CONFIG);
-    final URL configUrlFromClassLoader = getClass().getClassLoader().getResource(LogService.DEFAULT_CONFIG.substring(1));
-    final URL configUrlFromClassPathLoader = ClassPathLoader.getLatest().getResource(LogService.DEFAULT_CONFIG.substring(1));
-    
-    assertThat(configUrlFromLogService).isNotNull();
-    assertThat(configUrlFromClassLoader).isNotNull();
-    assertThat(configUrlFromClassPathLoader).isNotNull();
-    assertThat(configUrlFromLogService)
-        .isEqualTo(configUrlFromClassLoader)
-        .isEqualTo(configUrlFromClassPathLoader);
+    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).isNullOrEmpty();
   }
   
   @Test
@@ -167,43 +171,37 @@ public class LogServiceIntegrationJUnitTest {
   }
   
   @Test
-  public void intializeAfterUsingLoggerShouldReconfigure() {
-    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).as("log4j.configurationFile="+System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).isNullOrEmpty();
-    
-    Configurator.shutdown();
-    
-    LogManager.getRootLogger();
-
-    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).as("log4j.configurationFile="+System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).isNullOrEmpty();
+  public void shouldNotUseDefaultConfigIfCliConfigSpecified() throws Exception {
+    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, this.cliConfigUrl.toString());
     
     LogService.reconfigure();
-    LogService.initialize();
-    
-    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).as("log4j.configurationFile="+System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).contains(LogService.DEFAULT_CONFIG);
-    assertThat(LogService.isUsingGemFireDefaultConfig()).as(LogService.getConfigInformation()).isTrue();
-  }
   
-  @Test
-  public void cliConfigLoadsAsResource() {
-    assertThat(this.cliConfigUrl).isNotNull();
-    assertThat(this.cliConfigUrl.toString()).contains(LogService.CLI_CONFIG);
+    assertThat(LogService.isUsingGemFireDefaultConfig()).as(LogService.getConfigInformation()).isFalse();
+    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).isEqualTo(this.cliConfigUrl.toString());
+    assertThat(LogService.getLogger().getName()).isEqualTo(getClass().getName());
   }
-  
+
   @Test
-  public void defaultConfigLoadsAsResource() {
-    assertThat(this.defaultConfigUrl).isNotNull();
-    assertThat(this.defaultConfigUrl.toString()).contains(LogService.DEFAULT_CONFIG);
+  public void isUsingGemFireDefaultConfigShouldBeTrueIfDefaultConfig() throws Exception {
+    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, this.defaultConfigUrl.toString());
+    
+    assertThat(LogService.getConfiguration().getConfigurationSource().toString()).contains(DEFAULT_CONFIG_FILE_NAME);
+    assertThat(LogService.isUsingGemFireDefaultConfig()).isTrue();
   }
-  
+
   @Test
-  public void shouldConvertConfigurationFilePropertyValueToURL() throws Exception {
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, LogService.CLI_CONFIG);
+  public void isUsingGemFireDefaultConfigShouldBeFalseIfCliConfig() throws Exception {
+    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, this.cliConfigUrl.toString());
     
-    LogService.reconfigure();
+    assertThat(LogService.getConfiguration().getConfigurationSource().toString()).doesNotContain(DEFAULT_CONFIG_FILE_NAME);
+    assertThat(LogService.isUsingGemFireDefaultConfig()).isFalse();
+  }
+
+  @Test
+  public void shouldUseCliConfigIfCliConfigIsSpecifiedViaClasspath() throws Exception {
+    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, "classpath:"+CLI_CONFIG_FILE_NAME);
     
-    assertThat(LogService.isUsingGemFireDefaultConfig()).as(LogService.getConfigInformation()).isFalse();
-    assertThat(this.cliConfigUrl.toString()).contains(LogService.CLI_CONFIG);
-    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).isEqualTo(this.cliConfigUrl.toString());
-    assertThat(LogService.getLogger().getName()).isEqualTo(getClass().getName());
+    assertThat(LogService.getConfiguration().getConfigurationSource().toString()).contains(CLI_CONFIG_FILE_NAME);
+    assertThat(LogService.isUsingGemFireDefaultConfig()).isFalse();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java
index 8d6278a..c8c7eda 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceJUnitTest.java
@@ -3,14 +3,17 @@ package com.gemstone.gemfire.internal.logging;
 import static junitparams.JUnitParamsRunner.$;
 import static org.assertj.core.api.Assertions.*;
 
+import java.net.URL;
+
 import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.RestoreSystemProperties;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 
+import com.gemstone.gemfire.internal.ClassPathLoader;
 import com.gemstone.gemfire.internal.logging.log4j.AppenderContext;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
@@ -29,6 +32,15 @@ public class LogServiceJUnitTest {
   @Rule
   public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
   
+  private URL defaultConfigUrl;
+  private URL cliConfigUrl;
+  
+  @Before
+  public void setUp() {
+    this.defaultConfigUrl = LogService.class.getResource(LogService.DEFAULT_CONFIG);
+    this.cliConfigUrl = LogService.class.getResource(LogService.CLI_CONFIG);
+  }
+  
   @Test
   public void getAppenderContextShouldHaveEmptyName() throws Exception {
     final AppenderContext appenderContext = LogService.getAppenderContext();
@@ -45,39 +57,37 @@ public class LogServiceJUnitTest {
   }
   
   @Test
-  public void isUsingGemFireDefaultConfigShouldBeTrueIfDefaultConfig() {
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, LogService.DEFAULT_CONFIG);
-    
-    assertThat(LogService.isUsingGemFireDefaultConfig()).isTrue();
+  @Parameters(method = "getToLevelParameters")
+  public void toLevelShouldReturnMatchingLog4jLevel(final int intLevel, final Level level) {
+    assertThat(LogService.toLevel(intLevel)).isSameAs(level);
   }
 
   @Test
-  public void isUsingGemFireDefaultConfigShouldBeFalseIfEmpty() {
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, "");
-    
-    assertThat(LogService.isUsingGemFireDefaultConfig()).isFalse();
+  public void cliConfigLoadsAsResource() {
+    assertThat(this.cliConfigUrl).isNotNull();
+    assertThat(this.cliConfigUrl.toString()).contains(LogService.CLI_CONFIG);
   }
-
+  
   @Test
-  public void isUsingGemFireDefaultConfigShouldBeFalseIfNull() {
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, "");
-    
-    assertThat(LogService.isUsingGemFireDefaultConfig()).isFalse();
+  public void defaultConfigLoadsAsResource() {
+    assertThat(this.defaultConfigUrl).isNotNull();
+    assertThat(this.defaultConfigUrl.toString()).contains(LogService.DEFAULT_CONFIG);
   }
-
+  
   @Test
-  public void isUsingGemFireDefaultConfigShouldBeFalseIfCliConfig() {
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, LogService.CLI_CONFIG);
+  public void defaultConfigShouldBeLoadableAsResource() {
+    final URL configUrlFromLogService = LogService.class.getResource(LogService.DEFAULT_CONFIG);
+    final URL configUrlFromClassLoader = getClass().getClassLoader().getResource(LogService.DEFAULT_CONFIG.substring(1));
+    final URL configUrlFromClassPathLoader = ClassPathLoader.getLatest().getResource(LogService.DEFAULT_CONFIG.substring(1));
     
-    assertThat(LogService.isUsingGemFireDefaultConfig()).isFalse();
+    assertThat(configUrlFromLogService).isNotNull();
+    assertThat(configUrlFromClassLoader).isNotNull();
+    assertThat(configUrlFromClassPathLoader).isNotNull();
+    assertThat(configUrlFromLogService)
+        .isEqualTo(configUrlFromClassLoader)
+        .isEqualTo(configUrlFromClassPathLoader);
   }
   
-  @Test
-  @Parameters(method = "getToLevelParameters")
-  public void toLevelShouldReturnMatchingLog4jLevel(final int intLevel, final Level level) {
-    assertThat(LogService.toLevel(intLevel)).isSameAs(level);
-  }
-
   @SuppressWarnings("unused")
   private static final Object[] getToLevelParameters() {
     return $(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceUserDirIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceUserDirIntegrationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceUserDirIntegrationJUnitTest.java
deleted file mode 100755
index 4d83c22..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LogServiceUserDirIntegrationJUnitTest.java
+++ /dev/null
@@ -1,70 +0,0 @@
-package com.gemstone.gemfire.internal.logging;
-
-import static com.gemstone.gemfire.internal.logging.LogServiceIntegrationTestSupport.*;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assume.assumeFalse;
-
-import java.io.File;
-import java.net.MalformedURLException;
-
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.core.config.ConfigurationFactory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.contrib.java.lang.system.ClearSystemProperties;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.ExternalResource;
-
-import com.gemstone.gemfire.internal.lang.SystemUtils;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-/**
- * Tests LogService when a log4j2 config file is in the user.dir
- * 
- * @author Kirk Lund
- */
-@Category(IntegrationTest.class)
-public class LogServiceUserDirIntegrationJUnitTest {
-
-  @Rule
-  public final ClearSystemProperties clearConfigFileProperty = new ClearSystemProperties(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
-  
-  @Rule
-  public final ExternalResource externalResource = new ExternalResource() {
-    @Override
-    protected void after() {
-      LogService.reconfigure();
-      assertThat(LogService.isUsingGemFireDefaultConfig()).as(LogService.getConfigInformation()).isTrue();
-    }
-  };
-  
-  private File configFile;
-  
-  @Before
-  public void setUp() throws Exception {
-    this.configFile = new File(System.getProperty("user.dir"), "log4j2-test.xml");
-    writeConfigFile(this.configFile, Level.DEBUG);
-    LogService.reconfigure();
-  }
-  
-  @After
-  public void tearDown() {
-    this.configFile.delete();
-  }
-  
-  @Test
-  public void shouldPreferConfigInCurrentDirectoryIfFound() throws Exception {
-    // if working directory is in classpath this test will fail
-    assumeFalse(isUserDirInClassPath());
-    
-    assertThat(LogService.isUsingGemFireDefaultConfig()).as(LogService.getConfigInformation()).isFalse();
-    //ConfigurationFactory.getInstance().getConfiguration(null, null); TODO: delete
-    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)).isEqualTo(this.configFile.toURI().toString());
-  }
-  
-  private static boolean isUserDirInClassPath() throws MalformedURLException {
-    return SystemUtils.isInClassPath(System.getProperty("user.dir"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingIntegrationTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingIntegrationTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingIntegrationTestSuite.java
index dd9504b..cee05c3 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingIntegrationTestSuite.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingIntegrationTestSuite.java
@@ -3,13 +3,17 @@ package com.gemstone.gemfire.internal.logging;
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 
+import com.gemstone.gemfire.internal.logging.log4j.FastLoggerIntegrationJUnitTest;
+import com.gemstone.gemfire.internal.logging.log4j.FastLoggerWithDefaultConfigJUnitTest;
+
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
   DistributedSystemLogFileJUnitTest.class,
   LocatorLogFileJUnitTest.class,
   LogServiceIntegrationJUnitTest.class,
-  LogServiceUserDirIntegrationJUnitTest.class,
   MergeLogFilesJUnitTest.class,
+  FastLoggerWithDefaultConfigJUnitTest.class,
+  FastLoggerIntegrationJUnitTest.class,
 })
 public class LoggingIntegrationTestSuite {
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingUnitTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingUnitTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingUnitTestSuite.java
index 6b7b73e..ec3e04a 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingUnitTestSuite.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingUnitTestSuite.java
@@ -3,11 +3,24 @@ package com.gemstone.gemfire.internal.logging;
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 
+import com.gemstone.gemfire.internal.logging.log4j.AlertAppenderJUnitTest;
+import com.gemstone.gemfire.internal.logging.log4j.ConfigLocatorJUnitTest;
+import com.gemstone.gemfire.internal.logging.log4j.FastLoggerJUnitTest;
+import com.gemstone.gemfire.internal.logging.log4j.FastLoggerWithDefaultConfigJUnitTest;
+import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessageJUnitTest;
+import com.gemstone.gemfire.internal.logging.log4j.LogWriterAppenderJUnitTest;
+
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
   LogServiceJUnitTest.class,
   LogWriterImplJUnitTest.class,
-  SortLogFileJUnitTest.class
+  SortLogFileJUnitTest.class,
+  AlertAppenderJUnitTest.class,
+  ConfigLocatorJUnitTest.class,
+  FastLoggerJUnitTest.class,
+  FastLoggerWithDefaultConfigJUnitTest.class,
+  LocalizedMessageJUnitTest.class,
+  LogWriterAppenderJUnitTest.class,
 })
 public class LoggingUnitTestSuite {
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/MergeLogFilesJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/MergeLogFilesJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/MergeLogFilesJUnitTest.java
index d8d774c..37ba6c2 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/MergeLogFilesJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/MergeLogFilesJUnitTest.java
@@ -90,7 +90,7 @@ public class MergeLogFilesJUnitTest {
     PrintWriter pw = new PrintWriter(sw, true);
     MergeLogFiles.mergeLogFiles(streams, names, pw);
 
-    System.out.println(sw.toString());
+    //System.out.println(sw.toString());
 
     // Verfiy that the entries are sorted
     BufferedReader br =
@@ -222,7 +222,7 @@ public class MergeLogFilesJUnitTest {
         }
       }
 
-      System.out.println(baos.toString());
+      //System.out.println(baos.toString());
       in = new ByteArrayInputStream(baos.toByteArray());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerIntegrationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerIntegrationJUnitTest.java
index f1c7ab9..c20b6a7 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerIntegrationJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerIntegrationJUnitTest.java
@@ -47,6 +47,7 @@ public class FastLoggerIntegrationJUnitTest {
   public final TemporaryFolder temporaryFolder = new TemporaryFolder();
   
   private File configFile;
+  private String configFileLocation;
   private Logger logger;
   private LoggerContext appenderContext;
   private Marker enabledMarker;
@@ -56,6 +57,7 @@ public class FastLoggerIntegrationJUnitTest {
   public void setUp() throws Exception {
     System.clearProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
     this.configFile = new File(this.temporaryFolder.getRoot(), "log4j2-test.xml");
+    this.configFileLocation = this.configFile.toURI().toURL().toString();
     this.enabledMarker = MarkerManager.getMarker(ENABLED_MARKER_NAME);
     this.unusedMarker = MarkerManager.getMarker(UNUSED_MARKER_NAME);
     setUpLogService();
@@ -70,7 +72,7 @@ public class FastLoggerIntegrationJUnitTest {
   private void setUpLogService() throws Exception {
     // Load a base config and do some sanity checks
     writeSimpleConfigFile(this.configFile, Level.WARN);
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, this.configFile.toURI().toURL().toString());
+    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, this.configFileLocation);
 
     LogService.reconfigure();
     LogService.getLogger().getName(); // This causes the config file to be loaded

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/21743ec1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerWithDefaultConfigJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerWithDefaultConfigJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerWithDefaultConfigJUnitTest.java
index 2e08500..7069d20 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerWithDefaultConfigJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerWithDefaultConfigJUnitTest.java
@@ -1,6 +1,6 @@
 package com.gemstone.gemfire.internal.logging.log4j;
 
-import static org.hamcrest.CoreMatchers.*;
+import static org.hamcrest.Matchers.*;
 import static org.junit.Assert.*;
 
 import org.apache.logging.log4j.Logger;
@@ -41,7 +41,7 @@ public class FastLoggerWithDefaultConfigJUnitTest {
    */
   @Test
   public void configurationFilePropertyIsDefaultConfig() {
-    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY), containsString(LogService.DEFAULT_CONFIG));
+    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY), isEmptyOrNullString());
   }
   
   /**



[07/38] incubator-geode git commit: [GEODE-338] Modified FDDUnitTest, increased thread count and added logging on failure

Posted by bs...@apache.org.
[GEODE-338] Modified FDDUnitTest, increased thread count and added logging on failure


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

Branch: refs/heads/feature/GEODE-77
Commit: 28101725c5e882eafd1592df31f552716e3effa2
Parents: 1512677
Author: Jason Huynh <jh...@pivotal.io>
Authored: Mon Oct 5 10:02:47 2015 -0700
Committer: Jason Huynh <jh...@pivotal.io>
Committed: Mon Oct 5 10:05:04 2015 -0700

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/internal/FDDUnitTest.java | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/28101725/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
index 9d3bcb9..7379c86 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
@@ -81,9 +81,10 @@ public class FDDUnitTest extends CacheTestCase {
       }
     }
     try {
+      StringBuffer incaseOfFailure = new StringBuffer();
       final int[] port = AvailablePortHelper.getRandomAvailableTCPPorts(3);
       final int mcastPort = AvailablePortHelper.getRandomAvailableUDPPort();
-      int numThreads = 10;
+      int numThreads = 30;
 
       startCacheServer(vm0, port[0], mcastPort);
       startCacheServer(vm1, port[1], mcastPort);
@@ -99,6 +100,7 @@ public class FDDUnitTest extends CacheTestCase {
       doPuts(vm0, numThreads, "portfolios");
       long endFDs = checkFD(vm0);
       long numFDs = endFDs - startingFDs;
+      incaseOfFailure.append("NoSelectorPooling startFDs: " + startingFDs + " endFDs: " + endFDs + " diff:" + numFDs + " ");
 
       // run test with selector pooling
       setUseSelectorPooling(vm0, true);
@@ -106,7 +108,8 @@ public class FDDUnitTest extends CacheTestCase {
       doPuts(vm0, numThreads, "portfolios");
       long endFDsWithPooling = checkFD(vm0);
       long numFDsWithPooling = endFDsWithPooling - startingFDsWithPooling;
-      assertTrue(numFDsWithPooling < numFDs);
+      incaseOfFailure.append("SelectorPooling#1 startFDs: " + startingFDsWithPooling + " endFDs: " + endFDsWithPooling + " diff:" + numFDsWithPooling + " ");
+      assertTrue(incaseOfFailure.toString(), numFDsWithPooling < numFDs);
 
       // run it again and see if the number still is below
       startingFDsWithPooling = checkFD(vm0);
@@ -115,7 +118,8 @@ public class FDDUnitTest extends CacheTestCase {
       numFDsWithPooling = endFDsWithPooling - startingFDsWithPooling;
       // if you see these asserts failing, it could be that we are not using the
       // selector pool
-      assertTrue(numFDsWithPooling < numFDs);
+      incaseOfFailure.append("SelectorPooling#2 startFDs: " + startingFDsWithPooling + " endFDs: " + endFDsWithPooling + " diff:" + numFDsWithPooling + " ");
+      assertTrue(incaseOfFailure.toString(), numFDsWithPooling < numFDs);
 
     } finally {
       setUseSelectorPooling(vm0, true);
@@ -151,7 +155,7 @@ public class FDDUnitTest extends CacheTestCase {
           for (int i = 0; i < numThreads; i++) {
             executor.execute(new Runnable() {
               public void run() {
-                  for (int i = 0; i < 20; i++) {
+                  for (int i = 0; i < 10; i++) {
                     String myValue = "string" + i;
                     region.put("k" + i, myValue);
                     try {


[36/38] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-77

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/ParentLocalizedStrings.java
index d3ea781,44e67ca..7eba6b6
--- 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
@@@ -27,1074 -28,1068 +27,1068 @@@ class ParentLocalizedStrings 
  
    //alias for use in the test tree
    public static final StringId TESTING = ONE_ARG;
 -  public static final StringId TWO_ARG = new StringIdImpl(2, "{0} {1}");
 -  public static final StringId TWO_ARG_COLON = new StringIdImpl(3, "{0} : {1}");
 -  public static final StringId ERROR = new StringIdImpl(4, "ERROR");
 -  public static final StringId CACHE_IS_CLOSING = new StringIdImpl(1021, "Cache is closing");
 -  public static final StringId NOT_A_REAL_GEMFIREVM = new StringIdImpl(1022, "Not a real GemFireVM");
 -  public static final StringId SHOULDNT_INVOKE = new StringIdImpl(1023, "Should not be invoked");
 -  public static final StringId UNSUPPORTED_AT_THIS_TIME = new StringIdImpl(1024, "Unsupported at this time");
 +  public static final StringId TWO_ARG = new StringId(2, "{0} {1}");
 +  public static final StringId TWO_ARG_COLON = new StringId(3, "{0} : {1}");
 +  public static final StringId ERROR = new StringId(4, "ERROR");
 +  public static final StringId CACHE_IS_CLOSING = new StringId(1021, "Cache is closing");
 +  public static final StringId NOT_A_REAL_GEMFIREVM = new StringId(1022, "Not a real GemFireVM");
 +  public static final StringId SHOULDNT_INVOKE = new StringId(1023, "Should not be invoked");
 +  public static final StringId UNSUPPORTED_AT_THIS_TIME = new StringId(1024, "Unsupported at this time");
  
    /**Gemfire strings, messageId 1025-15000 **/
 -  public static final StringId AbstractHealthEvaluator_OKAY_HEALTH__0 = new StringIdImpl(1025, "OKAY_HEALTH:  {0}");
 -  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");
 +  public static final StringId AbstractHealthEvaluator_OKAY_HEALTH__0 = new StringId(1025, "OKAY_HEALTH:  {0}");
 +  public static final StringId AbstractHealthEvaluator_POOR_HEALTH__0 = new StringId(1026, "POOR_HEALTH:  {0}");
 +  public static final StringId AbstractRegion_CACHECALLBACK_CLOSE_EXCEPTION = new StringId(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 StringId(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 StringId(1030, "Bridge server: failed accepting client connection due to socket timeout.");
-   public static final StringId AcceptorImpl_BRIDGE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION__0 = new StringId(1031, "Bridge server: failed accepting client connection  {0}");
-   public static final StringId AcceptorImpl_BRIDGE_SERVER_ON_PORT_0_IS_SHUTTING_DOWN = new StringId(1032, "Bridge server on port {0} is shutting down.");
-   public static final StringId AcceptorImpl_BRIDGE_SERVER_TIMED_OUT_WAITING_FOR_HANDSHAKE_FROM__0 = new StringId(1033, "Bridge server: timed out waiting for handshake from  {0}");
-   public static final StringId AcceptorImpl_BRIDGE_SERVER_UNEXPECTED_EXCEPTION = new StringId(1034, "Bridge server: Unexpected Exception");
-   public static final StringId AcceptorImpl_BRIDGE_SERVER_UNEXPECTED_IOEXCEPTION_FROM_ACCEPT = new StringId(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}");
 -  public static final StringId CacheClientNotifier_CACHECLIENTNOTIFIER_A_PREVIOUS_CONNECTION_ATTEMPT_FROM_THIS_CLIENT_IS_STILL_BEING_PROCESSED__0 = new StringIdImpl(1039, "A previous connection attempt from this client is still being processed: {0}");
 -  public static final StringId AcceptorImpl_REJECTED_CONNECTION_FROM_0_BECAUSE_CURRENT_CONNECTION_COUNT_OF_1_IS_GREATER_THAN_OR_EQUAL_TO_THE_CONFIGURED_MAX_OF_2 = new StringIdImpl(1040, "Rejected connection from {0} because current connection count of {1} is greater than or equal to the configured max of {2}");
 -  public static final StringId AcceptorImpl_SELECTOR_ENABLED = new StringIdImpl(1041, "SELECTOR enabled");
 -  public static final StringId AcceptorImpl_UNEXPECTED = new StringIdImpl(1042, "unexpected");
 -  public static final StringId AdminDistributedSystem_COULD_NOT_SET_THE_GEMFIRE_VM = new StringIdImpl(1043, "Could not set the GemFire VM.");
 -  public static final StringId AdminDistributedSystemImpl_ADDING_NEW_APPLICATION_FOR__0 = new StringIdImpl(1044, "Adding new Application for  {0}");
 -  public static final StringId AdminDistributedSystemImpl_ADDING_NEW_CACHESERVER_FOR__0 = new StringIdImpl(1045, "Adding new CacheServer for  {0}");
 -  public static final StringId AdminDistributedSystemImpl_WHILE_GETTING_CANONICAL_FILE = new StringIdImpl(1046, "While getting canonical file");
 -  public static final StringId AdminRequest_RESPONSE_TO__0__WAS_CANCELLED = new StringIdImpl(1047, "Response to  {0}  was cancelled.");
 -  public static final StringId AdminWaiters_COULD_NOT_SEND_REQUEST_0 = new StringIdImpl(1048, "Could not send request.{0}");
 -  public static final StringId AdminWaiters_REQUEST_SEND_TO_0_WAS_CANCELLED_1 = new StringIdImpl(1049, "Request sent to {0} was cancelled. {1}");
 -  public static final StringId AdminWaiters_REQUEST_SENT_TO_0_FAILED_SINCE_MEMBER_DEPARTED_1 = new StringIdImpl(1050, "Request sent to {0} failed since member departed.{1}");
 -  public static final StringId AgentImpl_0__IS_ALREADY_REGISTERED = new StringIdImpl(1051, "{0}  is already registered.");
 -  public static final StringId AgentImpl_AGENT_HAS_STOPPED = new StringIdImpl(1052, "Agent has stopped");
 -  public static final StringId AgentImpl_AUTO_CONNECT_FAILED__0 = new StringIdImpl(1053, "auto connect failed:  {0}");
 -  public static final StringId AgentImpl_FAILED_TO_START_HTTPADAPTOR__0 = new StringIdImpl(1054, "Failed to start HttpAdaptor:  {0}");
 -  public static final StringId AgentImpl_FAILED_TO_START_RMICONNECTORSERVER = new StringIdImpl(1055, "Failed to start RMIConnectorServer:");
 -  public static final StringId AgentImpl_FAILED_TO_START_SNMPADAPTOR__0 = new StringIdImpl(1056, "Failed to start SnmpAdaptor:  {0}");
 -  public static final StringId AgentImpl_GEMFIRE_JMX_AGENT_IS_RUNNING = new StringIdImpl(1057, "GemFire JMX Agent is running...");
 -  public static final StringId AgentImpl_HTTPADAPTOR_ALREADY_REGISTERED_AS__0 = new StringIdImpl(1058, "HttpAdaptor already registered as  {0}");
 -  public static final StringId AgentImpl_HTTP_ADAPTOR_LISTENING_ON_ADDRESS__0 = new StringIdImpl(1059, "HTTP adaptor listening on address:  {0}");
 -  public static final StringId AgentImpl_HTTP_ADAPTOR_LISTENING_ON_PORT__0 = new StringIdImpl(1060, "HTTP adaptor listening on port:  {0}");
 -  public static final StringId AgentImpl_INCORRECT_NULL_HOSTNAME = new StringIdImpl(1061, "Incorrect null hostname");
 -  public static final StringId AgentImpl_INCORRECT_PORT_VALUE__0 = new StringIdImpl(1062, "Incorrect port value  {0}");
 -  public static final StringId AgentImpl_RMICONNECTORSERVER_ALREADY_REGISTERED_AS__0 = new StringIdImpl(1063, "RMIConnectorServer already registered as  {0}");
 -  public static final StringId AgentImpl_SNMPADAPTOR_ALREADY_REGISTERED_AS__0 = new StringIdImpl(1064, "SnmpAdaptor already registered as  {0}");
 -  public static final StringId AgentImpl_STOPPING_JMX_AGENT = new StringIdImpl(1065, "Stopping JMX agent");
 -  public static final StringId AgentImpl_XSLTPROCESSOR_ALREADY_REGISTERED_AS__0 = new StringIdImpl(1066, "XsltProcessor already registered as  {0}");
++  public static final StringId AcceptorImpl_CACHE_SERVER_CONNECTION_LISTENER_BOUND_TO_ADDRESS_0_WITH_BACKLOG_1 = new StringId(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 StringId(1030, "Cache server: failed accepting client connection due to socket timeout.");
++  public static final StringId AcceptorImpl_CACHE_SERVER_FAILED_ACCEPTING_CLIENT_CONNECTION__0 = new StringId(1031, "Cache server: failed accepting client connection  {0}");
++  public static final StringId AcceptorImpl_CACHE_SERVER_ON_PORT_0_IS_SHUTTING_DOWN = new StringId(1032, "Cache server on port {0} is shutting down.");
++  public static final StringId AcceptorImpl_CACHE_SERVER_TIMED_OUT_WAITING_FOR_HANDSHAKE_FROM__0 = new StringId(1033, "Cache server: timed out waiting for handshake from  {0}");
++  public static final StringId AcceptorImpl_CACHE_SERVER_UNEXPECTED_EXCEPTION = new StringId(1034, "Cache server: Unexpected Exception");
++  public static final StringId AcceptorImpl_CACHE_SERVER_UNEXPECTED_IOEXCEPTION_FROM_ACCEPT = new StringId(1035, "Cache server: Unexpected IOException from accept");
 +  public static final StringId AcceptorImpl_EXCEEDED_MAX_CONNECTIONS_0 = new StringId(1036, "exceeded max-connections {0}");
 +  public static final StringId AcceptorImpl_IGNORING = new StringId(1037, "ignoring");
 +  public static final StringId AcceptorImpl_IGNORING_EVENT_ON_SELECTOR_KEY__0 = new StringId(1038, "ignoring event on selector key  {0}");
 +  public static final StringId CacheClientNotifier_CACHECLIENTNOTIFIER_A_PREVIOUS_CONNECTION_ATTEMPT_FROM_THIS_CLIENT_IS_STILL_BEING_PROCESSED__0 = new StringId(1039, "A previous connection attempt from this client is still being processed: {0}");
 +  public static final StringId AcceptorImpl_REJECTED_CONNECTION_FROM_0_BECAUSE_CURRENT_CONNECTION_COUNT_OF_1_IS_GREATER_THAN_OR_EQUAL_TO_THE_CONFIGURED_MAX_OF_2 = new StringId(1040, "Rejected connection from {0} because current connection count of {1} is greater than or equal to the configured max of {2}");
 +  public static final StringId AcceptorImpl_SELECTOR_ENABLED = new StringId(1041, "SELECTOR enabled");
 +  public static final StringId AcceptorImpl_UNEXPECTED = new StringId(1042, "unexpected");
 +  public static final StringId AdminDistributedSystem_COULD_NOT_SET_THE_GEMFIRE_VM = new StringId(1043, "Could not set the GemFire VM.");
 +  public static final StringId AdminDistributedSystemImpl_ADDING_NEW_APPLICATION_FOR__0 = new StringId(1044, "Adding new Application for  {0}");
 +  public static final StringId AdminDistributedSystemImpl_ADDING_NEW_CACHESERVER_FOR__0 = new StringId(1045, "Adding new CacheServer for  {0}");
 +  public static final StringId AdminDistributedSystemImpl_WHILE_GETTING_CANONICAL_FILE = new StringId(1046, "While getting canonical file");
 +  public static final StringId AdminRequest_RESPONSE_TO__0__WAS_CANCELLED = new StringId(1047, "Response to  {0}  was cancelled.");
 +  public static final StringId AdminWaiters_COULD_NOT_SEND_REQUEST_0 = new StringId(1048, "Could not send request.{0}");
 +  public static final StringId AdminWaiters_REQUEST_SEND_TO_0_WAS_CANCELLED_1 = new StringId(1049, "Request sent to {0} was cancelled. {1}");
 +  public static final StringId AdminWaiters_REQUEST_SENT_TO_0_FAILED_SINCE_MEMBER_DEPARTED_1 = new StringId(1050, "Request sent to {0} failed since member departed.{1}");
 +  public static final StringId AgentImpl_0__IS_ALREADY_REGISTERED = new StringId(1051, "{0}  is already registered.");
 +  public static final StringId AgentImpl_AGENT_HAS_STOPPED = new StringId(1052, "Agent has stopped");
 +  public static final StringId AgentImpl_AUTO_CONNECT_FAILED__0 = new StringId(1053, "auto connect failed:  {0}");
 +  public static final StringId AgentImpl_FAILED_TO_START_HTTPADAPTOR__0 = new StringId(1054, "Failed to start HttpAdaptor:  {0}");
 +  public static final StringId AgentImpl_FAILED_TO_START_RMICONNECTORSERVER = new StringId(1055, "Failed to start RMIConnectorServer:");
 +  public static final StringId AgentImpl_FAILED_TO_START_SNMPADAPTOR__0 = new StringId(1056, "Failed to start SnmpAdaptor:  {0}");
 +  public static final StringId AgentImpl_GEMFIRE_JMX_AGENT_IS_RUNNING = new StringId(1057, "GemFire JMX Agent is running...");
 +  public static final StringId AgentImpl_HTTPADAPTOR_ALREADY_REGISTERED_AS__0 = new StringId(1058, "HttpAdaptor already registered as  {0}");
 +  public static final StringId AgentImpl_HTTP_ADAPTOR_LISTENING_ON_ADDRESS__0 = new StringId(1059, "HTTP adaptor listening on address:  {0}");
 +  public static final StringId AgentImpl_HTTP_ADAPTOR_LISTENING_ON_PORT__0 = new StringId(1060, "HTTP adaptor listening on port:  {0}");
 +  public static final StringId AgentImpl_INCORRECT_NULL_HOSTNAME = new StringId(1061, "Incorrect null hostname");
 +  public static final StringId AgentImpl_INCORRECT_PORT_VALUE__0 = new StringId(1062, "Incorrect port value  {0}");
 +  public static final StringId AgentImpl_RMICONNECTORSERVER_ALREADY_REGISTERED_AS__0 = new StringId(1063, "RMIConnectorServer already registered as  {0}");
 +  public static final StringId AgentImpl_SNMPADAPTOR_ALREADY_REGISTERED_AS__0 = new StringId(1064, "SnmpAdaptor already registered as  {0}");
 +  public static final StringId AgentImpl_STOPPING_JMX_AGENT = new StringId(1065, "Stopping JMX agent");
 +  public static final StringId AgentImpl_XSLTPROCESSOR_ALREADY_REGISTERED_AS__0 = new StringId(1066, "XsltProcessor already registered as  {0}");
    // ok to reuse 1067
 -  public static final StringId AttributesFactory_0_WAS_NOT_AN_EXISTING_DIRECTORY = new StringIdImpl(1068, "\"{0}\" was not an existing directory.");
 -  public static final StringId AttributesFactory_ADDCACHELISTENER_PARAMETER_WAS_NULL = new StringIdImpl(1069, "addCacheListener parameter was null");
 -  public static final StringId AttributesFactory_AN_EVICTION_CONTROLLER_WITH_LOCAL_DESTROY_EVICTION_ACTION_IS_INCOMPATIBLE_WITH_DISTRIBUTED_REPLICATION = new StringIdImpl(1070, "An Eviction Controller with local destroy eviction action is incompatible with distributed replication");
 -  public static final StringId AttributesFactory_CAN_NOT_SET_DIFFERENT_CACHE_LOADER_IN_REGIONATTRIBUTES_AND_PARTITIONATTRIBUTES = new StringIdImpl(1071, "Can not set different cache loader in RegionAttributes and PartitionAttributes.");
 -  public static final StringId AttributesFactory_CONCURRENCYLEVEL_MUST_BE_0 = new StringIdImpl(1072, "concurrencyLevel must be > 0");
 -  public static final StringId AttributesFactory_DATAPOLICY_MUST_NOT_BE_NULL = new StringIdImpl(1073, "dataPolicy must not be null");
 -  public static final StringId AttributesFactory_DATA_POLICIES_OTHER_THAN_0_ARE_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringIdImpl(1074, "Data policies other than {0} are not allowed in  partitioned regions.");
 -  public static final StringId AttributesFactory_DATA_POLICIES_OTHER_THAN_0_ARE_NOT_SUPPORTED_FOR_PARTITIONED_REGIONS = new StringIdImpl(1075, "Data policies other than {0} are not supported for Partitioned Regions");
 -  public static final StringId AttributesFactory_DATA_POLICY_0_IS_NOT_ALLOWED_FOR_A_PARTITIONED_REGION_DATAPOLICIES_OTHER_THAN_1_ARE_NOT_ALLOWED = new StringIdImpl(1076, "Data policy {0} is not allowed for a partitioned region. DataPolicies other than {1} are not allowed.");
 -  public static final StringId AttributesFactory_DIR_SIZE_CANNOT_BE_NEGATIVE_0 = new StringIdImpl(1077, "Dir size cannot be negative : {0}");
 -  public static final StringId AttributesFactory_EVICTION_IS_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringIdImpl(1078, "Eviction is not allowed in partitioned regions.");
 -  public static final StringId AttributesFactory_EXPIRATIONACTIONLOCAL_DESTROY_ON_THE_ENTRIES_IS_INCOMPATIBLE_WITH_DISTRIBUTED_REPLICATION = new StringIdImpl(1079, "ExpirationAction.LOCAL_DESTROY on the entries is incompatible with distributed replication");
 -  public static final StringId AttributesFactory_EXPIRATIONACTIONLOCAL_INVALIDATE_ON_THE_ENTRIES_IS_INCOMPATIBLE_WITH_DISTRIBUTED_REPLICATION = new StringIdImpl(1080, "ExpirationAction.LOCAL_INVALIDATE on the entries is incompatible with distributed replication");
 -  public static final StringId AttributesFactory_EXPIRATIONACTIONLOCAL_INVALIDATE_ON_THE_REGION_IS_INCOMPATIBLE_WITH_DISTRIBUTED_REPLICATION = new StringIdImpl(1081, "ExpirationAction.LOCAL_INVALIDATE on the region is incompatible with distributed replication");
 -  public static final StringId AttributesFactory_EXPIRATIONATTRIBUTES_FOR_REGIONIDLETIMEOUT_ARE_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringIdImpl(1082, "ExpirationAttributes for regionIdleTimeout are not allowed in partitioned regions.");
 -  public static final StringId AttributesFactory_EXPIRATIONATTRIBUTES_FOR_REGIONTIMETOLIVE_ARE_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringIdImpl(1083, "ExpirationAttributes for regionTimeToLive are not allowed in partitioned regions.");
 -  public static final StringId AttributesFactory_EXPIRATION_ATTRIBUTES_ARE_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringIdImpl(1084, "Expiration Attributes are not allowed in partitioned regions.");
 -  public static final StringId AttributesFactory_IDLETIMEOUT_MUST_NOT_BE_NULL = new StringIdImpl(1085, "idleTimeout must not be null");
 -  public static final StringId AttributesFactory_IF_THE_DATA_POLICY_IS_0_THEN_ENTRY_EXPIRATION_IS_NOT_ALLOWED = new StringIdImpl(1086, "If the data policy is {0} then entry expiration is not allowed.");
 -  public static final StringId AttributesFactory_IF_THE_DATA_POLICY_IS_0_THEN_EVICTION_IS_NOT_ALLOWED = new StringIdImpl(1087, "If the data policy is {0} then eviction is not allowed.");
 -  public static final StringId AttributesFactory_IF_THE_MEMBERSHIP_ATTRIBUTES_HAS_REQUIRED_ROLES_THEN_SCOPE_MUST_NOT_BE_LOCAL = new StringIdImpl(1088, "If the membership attributes has required roles then scope must not be LOCAL.");
 -  public static final StringId AttributesFactory_INITCACHELISTENERS_PARAMETER_HAD_A_NULL_ELEMENT = new StringIdImpl(1089, "initCacheListeners parameter had a null element");
 -  public static final StringId AttributesFactory_INITIALCAPACITY_MUST_BE_0 = new StringIdImpl(1090, "initialCapacity must be >= 0");
 -  public static final StringId AttributesFactory_KEYCONSTRAINT_MUST_NOT_BE_A_PRIMITIVE_TYPE = new StringIdImpl(1091, "keyConstraint must not be a primitive type");
 -  public static final StringId AttributesFactory_LOADFACTOR_MUST_BE_0_VALUE_IS_0 = new StringIdImpl(1092, "loadFactor must be > 0, value is {0}");
 -  public static final StringId AttributesFactory_MIRRORTYPE_MUST_NOT_BE_NULL = new StringIdImpl(1093, "mirrorType must not be null");
 -  public static final StringId AttributesFactory_MORE_THAN_ONE_CACHE_LISTENER_EXISTS = new StringIdImpl(1094, "More than one cache listener exists.");
 -  public static final StringId AttributesFactory_NO_MIRROR_TYPE_CORRESPONDS_TO_DATA_POLICY_0 = new StringIdImpl(1095, "No mirror type corresponds to data policy \"{0}\".");
 -  public static final StringId AttributesFactory_NUMBER_OF_DISKSIZES_IS_0_WHICH_IS_NOT_EQUAL_TO_NUMBER_OF_DISK_DIRS_WHICH_IS_1 = new StringIdImpl(1096, " Number of diskSizes is {0} which is not equal to number of disk Dirs which is {1}");
 -  public static final StringId AttributesFactory_PARTITIONATTRIBUTES_LOCALMAXMEMORY_MUST_NOT_BE_NEGATIVE = new StringIdImpl(1097, "PartitionAttributes localMaxMemory must not be negative.");
 -  public static final StringId AttributesFactory_SCOPETYPE_MUST_NOT_BE_NULL = new StringIdImpl(1098, "scopeType must not be null");
 -  public static final StringId AttributesFactory_SETLOCKGRANTERTRUE_IS_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringIdImpl(1099, "setLockGranter(true) is not allowed in Partitioned Regions.");
 -  public static final StringId AttributesFactory_SETTING_SCOPE_ON_A_PARTITIONED_REGIONS_IS_NOT_ALLOWED = new StringIdImpl(1100, "Setting Scope on a Partitioned Regions is not allowed.");
 -  public static final StringId AttributesFactory_STATISTICS_MUST_BE_ENABLED_FOR_EXPIRATION = new StringIdImpl(1101, "Statistics must be enabled for expiration");
 -  public static final StringId AttributesFactory_TIMETOLIVE_MUST_NOT_BE_NULL = new StringIdImpl(1102, "timeToLive must not be null");
 -  public static final StringId AttributesFactory_TOTAL_SIZE_OF_PARTITION_REGION_MUST_BE_0 = new StringIdImpl(1103, "Total size of partition region must be > 0.");
 -  public static final StringId AttributesFactory_VALUECONSTRAINT_MUST_NOT_BE_A_PRIMITIVE_TYPE = new StringIdImpl(1104, "valueConstraint must not be a primitive type");
 -  public static final StringId BaseCommand_0_CONNECTION_DISCONNECT_DETECTED_BY_EOF = new StringIdImpl(1105, "{0}: connection disconnect detected by EOF.");
 -  public static final StringId BaseCommand_0_EOFEXCEPTION_DURING_A_WRITE_OPERATION_ON_REGION__1_KEY_2_MESSAGEID_3 = new StringIdImpl(1106, "{0}: EOFException during a write operation on region : {1} key: {2} messageId: {3}");
 -  public static final StringId BaseCommand_0_QUERYSTRING_IS_1 = new StringIdImpl(1107, "{0} : QueryString is: {1}.");
 -  public static final StringId BaseCommand_0_UNEXPECTED_ERROR_ON_SERVER = new StringIdImpl(1108, "{0} : Unexpected Error on server");
 -  public static final StringId BaseCommand_0_UNEXPECTED_EXCEPTION = new StringIdImpl(1109, "{0}: Unexpected Exception");
 -  public static final StringId BaseCommand_0_UNEXPECTED_EXCEPTION_DURING_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringIdImpl(1110, "{0}: Unexpected Exception during operation on region: {1} key: {2} messageId: {3}");
 -  public static final StringId BaseCommand_0_UNEXPECTED_IOEXCEPTION = new StringIdImpl(1111, "{0}: Unexpected IOException: ");
 -  public static final StringId BaseCommand_0_UNEXPECTED_IOEXCEPTION_DURING_OPERATION_FOR_REGION_1_KEY_2_MESSID_3 = new StringIdImpl(1112, "{0}: Unexpected IOException during operation for region: {1} key: {2} messId: {3}");
 -  public static final StringId BaseCommand_0_UNEXPECTED_SHUTDOWNEXCEPTION = new StringIdImpl(1113, "{0}: Unexpected ShutdownException: ");
 -  public static final StringId BaseCommand_0_UNEXPECTED_SHUTDOWNEXCEPTION_DURING_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringIdImpl(1114, "{0}: Unexpected ShutdownException during operation on region: {1} key: {2} messageId: {3}");
 -  public static final StringId BaseCommand_0_UNEXPECTED_THREADINTERRUPTEDEXCEPTION = new StringIdImpl(1115, "{0}: Unexpected ThreadInterruptedException: ");
 -  public static final StringId BaseCommand_0_UNEXPECTED_THREADINTERRUPTEDEXCEPTION_DURING_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringIdImpl(1116, "{0}: Unexpected ThreadInterruptedException during operation on region: {1} key: {2} messageId: {3}");
 -  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 AttributesFactory_0_WAS_NOT_AN_EXISTING_DIRECTORY = new StringId(1068, "\"{0}\" was not an existing directory.");
 +  public static final StringId AttributesFactory_ADDCACHELISTENER_PARAMETER_WAS_NULL = new StringId(1069, "addCacheListener parameter was null");
 +  public static final StringId AttributesFactory_AN_EVICTION_CONTROLLER_WITH_LOCAL_DESTROY_EVICTION_ACTION_IS_INCOMPATIBLE_WITH_DISTRIBUTED_REPLICATION = new StringId(1070, "An Eviction Controller with local destroy eviction action is incompatible with distributed replication");
 +  public static final StringId AttributesFactory_CAN_NOT_SET_DIFFERENT_CACHE_LOADER_IN_REGIONATTRIBUTES_AND_PARTITIONATTRIBUTES = new StringId(1071, "Can not set different cache loader in RegionAttributes and PartitionAttributes.");
 +  public static final StringId AttributesFactory_CONCURRENCYLEVEL_MUST_BE_0 = new StringId(1072, "concurrencyLevel must be > 0");
 +  public static final StringId AttributesFactory_DATAPOLICY_MUST_NOT_BE_NULL = new StringId(1073, "dataPolicy must not be null");
 +  public static final StringId AttributesFactory_DATA_POLICIES_OTHER_THAN_0_ARE_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringId(1074, "Data policies other than {0} are not allowed in  partitioned regions.");
 +  public static final StringId AttributesFactory_DATA_POLICIES_OTHER_THAN_0_ARE_NOT_SUPPORTED_FOR_PARTITIONED_REGIONS = new StringId(1075, "Data policies other than {0} are not supported for Partitioned Regions");
 +  public static final StringId AttributesFactory_DATA_POLICY_0_IS_NOT_ALLOWED_FOR_A_PARTITIONED_REGION_DATAPOLICIES_OTHER_THAN_1_ARE_NOT_ALLOWED = new StringId(1076, "Data policy {0} is not allowed for a partitioned region. DataPolicies other than {1} are not allowed.");
 +  public static final StringId AttributesFactory_DIR_SIZE_CANNOT_BE_NEGATIVE_0 = new StringId(1077, "Dir size cannot be negative : {0}");
 +  public static final StringId AttributesFactory_EVICTION_IS_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringId(1078, "Eviction is not allowed in partitioned regions.");
 +  public static final StringId AttributesFactory_EXPIRATIONACTIONLOCAL_DESTROY_ON_THE_ENTRIES_IS_INCOMPATIBLE_WITH_DISTRIBUTED_REPLICATION = new StringId(1079, "ExpirationAction.LOCAL_DESTROY on the entries is incompatible with distributed replication");
 +  public static final StringId AttributesFactory_EXPIRATIONACTIONLOCAL_INVALIDATE_ON_THE_ENTRIES_IS_INCOMPATIBLE_WITH_DISTRIBUTED_REPLICATION = new StringId(1080, "ExpirationAction.LOCAL_INVALIDATE on the entries is incompatible with distributed replication");
 +  public static final StringId AttributesFactory_EXPIRATIONACTIONLOCAL_INVALIDATE_ON_THE_REGION_IS_INCOMPATIBLE_WITH_DISTRIBUTED_REPLICATION = new StringId(1081, "ExpirationAction.LOCAL_INVALIDATE on the region is incompatible with distributed replication");
 +  public static final StringId AttributesFactory_EXPIRATIONATTRIBUTES_FOR_REGIONIDLETIMEOUT_ARE_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringId(1082, "ExpirationAttributes for regionIdleTimeout are not allowed in partitioned regions.");
 +  public static final StringId AttributesFactory_EXPIRATIONATTRIBUTES_FOR_REGIONTIMETOLIVE_ARE_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringId(1083, "ExpirationAttributes for regionTimeToLive are not allowed in partitioned regions.");
 +  public static final StringId AttributesFactory_EXPIRATION_ATTRIBUTES_ARE_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringId(1084, "Expiration Attributes are not allowed in partitioned regions.");
 +  public static final StringId AttributesFactory_IDLETIMEOUT_MUST_NOT_BE_NULL = new StringId(1085, "idleTimeout must not be null");
 +  public static final StringId AttributesFactory_IF_THE_DATA_POLICY_IS_0_THEN_ENTRY_EXPIRATION_IS_NOT_ALLOWED = new StringId(1086, "If the data policy is {0} then entry expiration is not allowed.");
 +  public static final StringId AttributesFactory_IF_THE_DATA_POLICY_IS_0_THEN_EVICTION_IS_NOT_ALLOWED = new StringId(1087, "If the data policy is {0} then eviction is not allowed.");
 +  public static final StringId AttributesFactory_IF_THE_MEMBERSHIP_ATTRIBUTES_HAS_REQUIRED_ROLES_THEN_SCOPE_MUST_NOT_BE_LOCAL = new StringId(1088, "If the membership attributes has required roles then scope must not be LOCAL.");
 +  public static final StringId AttributesFactory_INITCACHELISTENERS_PARAMETER_HAD_A_NULL_ELEMENT = new StringId(1089, "initCacheListeners parameter had a null element");
 +  public static final StringId AttributesFactory_INITIALCAPACITY_MUST_BE_0 = new StringId(1090, "initialCapacity must be >= 0");
 +  public static final StringId AttributesFactory_KEYCONSTRAINT_MUST_NOT_BE_A_PRIMITIVE_TYPE = new StringId(1091, "keyConstraint must not be a primitive type");
 +  public static final StringId AttributesFactory_LOADFACTOR_MUST_BE_0_VALUE_IS_0 = new StringId(1092, "loadFactor must be > 0, value is {0}");
 +  public static final StringId AttributesFactory_MIRRORTYPE_MUST_NOT_BE_NULL = new StringId(1093, "mirrorType must not be null");
 +  public static final StringId AttributesFactory_MORE_THAN_ONE_CACHE_LISTENER_EXISTS = new StringId(1094, "More than one cache listener exists.");
 +  public static final StringId AttributesFactory_NO_MIRROR_TYPE_CORRESPONDS_TO_DATA_POLICY_0 = new StringId(1095, "No mirror type corresponds to data policy \"{0}\".");
 +  public static final StringId AttributesFactory_NUMBER_OF_DISKSIZES_IS_0_WHICH_IS_NOT_EQUAL_TO_NUMBER_OF_DISK_DIRS_WHICH_IS_1 = new StringId(1096, " Number of diskSizes is {0} which is not equal to number of disk Dirs which is {1}");
 +  public static final StringId AttributesFactory_PARTITIONATTRIBUTES_LOCALMAXMEMORY_MUST_NOT_BE_NEGATIVE = new StringId(1097, "PartitionAttributes localMaxMemory must not be negative.");
 +  public static final StringId AttributesFactory_SCOPETYPE_MUST_NOT_BE_NULL = new StringId(1098, "scopeType must not be null");
 +  public static final StringId AttributesFactory_SETLOCKGRANTERTRUE_IS_NOT_ALLOWED_IN_PARTITIONED_REGIONS = new StringId(1099, "setLockGranter(true) is not allowed in Partitioned Regions.");
 +  public static final StringId AttributesFactory_SETTING_SCOPE_ON_A_PARTITIONED_REGIONS_IS_NOT_ALLOWED = new StringId(1100, "Setting Scope on a Partitioned Regions is not allowed.");
 +  public static final StringId AttributesFactory_STATISTICS_MUST_BE_ENABLED_FOR_EXPIRATION = new StringId(1101, "Statistics must be enabled for expiration");
 +  public static final StringId AttributesFactory_TIMETOLIVE_MUST_NOT_BE_NULL = new StringId(1102, "timeToLive must not be null");
 +  public static final StringId AttributesFactory_TOTAL_SIZE_OF_PARTITION_REGION_MUST_BE_0 = new StringId(1103, "Total size of partition region must be > 0.");
 +  public static final StringId AttributesFactory_VALUECONSTRAINT_MUST_NOT_BE_A_PRIMITIVE_TYPE = new StringId(1104, "valueConstraint must not be a primitive type");
 +  public static final StringId BaseCommand_0_CONNECTION_DISCONNECT_DETECTED_BY_EOF = new StringId(1105, "{0}: connection disconnect detected by EOF.");
 +  public static final StringId BaseCommand_0_EOFEXCEPTION_DURING_A_WRITE_OPERATION_ON_REGION__1_KEY_2_MESSAGEID_3 = new StringId(1106, "{0}: EOFException during a write operation on region : {1} key: {2} messageId: {3}");
 +  public static final StringId BaseCommand_0_QUERYSTRING_IS_1 = new StringId(1107, "{0} : QueryString is: {1}.");
 +  public static final StringId BaseCommand_0_UNEXPECTED_ERROR_ON_SERVER = new StringId(1108, "{0} : Unexpected Error on server");
 +  public static final StringId BaseCommand_0_UNEXPECTED_EXCEPTION = new StringId(1109, "{0}: Unexpected Exception");
 +  public static final StringId BaseCommand_0_UNEXPECTED_EXCEPTION_DURING_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringId(1110, "{0}: Unexpected Exception during operation on region: {1} key: {2} messageId: {3}");
 +  public static final StringId BaseCommand_0_UNEXPECTED_IOEXCEPTION = new StringId(1111, "{0}: Unexpected IOException: ");
 +  public static final StringId BaseCommand_0_UNEXPECTED_IOEXCEPTION_DURING_OPERATION_FOR_REGION_1_KEY_2_MESSID_3 = new StringId(1112, "{0}: Unexpected IOException during operation for region: {1} key: {2} messId: {3}");
 +  public static final StringId BaseCommand_0_UNEXPECTED_SHUTDOWNEXCEPTION = new StringId(1113, "{0}: Unexpected ShutdownException: ");
 +  public static final StringId BaseCommand_0_UNEXPECTED_SHUTDOWNEXCEPTION_DURING_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringId(1114, "{0}: Unexpected ShutdownException during operation on region: {1} key: {2} messageId: {3}");
 +  public static final StringId BaseCommand_0_UNEXPECTED_THREADINTERRUPTEDEXCEPTION = new StringId(1115, "{0}: Unexpected ThreadInterruptedException: ");
 +  public static final StringId BaseCommand_0_UNEXPECTED_THREADINTERRUPTEDEXCEPTION_DURING_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringId(1116, "{0}: Unexpected ThreadInterruptedException during operation on region: {1} key: {2} messageId: {3}");
 +  public static final StringId BaseCommand_UNKNOWN_QUERY_EXCEPTION = new StringId(1117, "Uknown query Exception.");
 +  public static final StringId BaseCommand_SEVERE_CACHE_EXCEPTION_0 = new StringId(1118, "Severe cache exception : {0}");
 +  public static final StringId BaseCommand_UNEXPECTED_QUERYINVALIDEXCEPTION_WHILE_PROCESSING_QUERY_0 = new StringId(1119, "Unexpected QueryInvalidException while processing query {0}");
-   public static final StringId BridgeLoader_THE_BRIDGELOADER_COULD_NOT_ACQUIRE_OR_CREATE_A_CONNECTION_BECAUSE__0 = new StringId(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 StringId(1121, "The Server is unreachable, could not connect after {0} attempts.");
-   public static final StringId BridgeServerImpl_CACHESERVER_CONFIGURATION___0 = new StringId(1122, "CacheServer Configuration:   {0}");
-   public static final StringId BridgeServerImpl_FORCING_NOTIFYBYSUBSCRIPTION_TO_SUPPORT_DYNAMIC_REGIONS = new StringId(1123, "Forcing notifyBySubscription to support dynamic regions");
-   public static final StringId BridgeServerImpl_STARTED__0 = new StringId(1124, "Started  {0}");
-   public static final StringId BridgeWriter_0_NO_ACTIVE_SERVERS_WERE_FOUND = new StringId(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 StringId(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 StringId(1128, "{0}: The BridgeWriter has been closed.");
-   public static final StringId BridgeWriter_THE_BRIDGEWRCLEARITER_COULD_NOT_ACQUIRE_OR_CREATE_A_CONNECTION_BECAUSE_0 = new StringId(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 StringId(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");
++  public static final StringId CacheServerImpl_CACHESERVER_CONFIGURATION___0 = new StringId(1122, "CacheServer Configuration:   {0}");
++  public static final StringId CacheServerImpl_FORCING_NOTIFYBYSUBSCRIPTION_TO_SUPPORT_DYNAMIC_REGIONS = new StringId(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.");
 -  public static final StringId BucketAdvisor_BUCKETADVISOR_WAS_NOT_CLOSED_PROPERLY = new StringIdImpl(1134, "BucketAdvisor was not closed properly.");
 -  public static final StringId BucketBackupMessage_BUCKETBACKUPMESSAGE_DATA_STORE_NOT_CONFIGURED_FOR_THIS_MEMBER = new StringIdImpl(1135, "BucketBackupMessage: data store not configured for this member");
 -  public static final StringId BucketCleanupMessage_BUCKETCLEANUPMESSAGE_DATA_STORE_NOT_CONFIGURED_FOR_THIS_MEMBER = new StringIdImpl(1136, "BucketCleanupMessage: data store not configured for this member");
 -  public static final StringId BucketRegion_FAILED_TO_SEND_OLD_VALUE_TO_THE_FOLLOWING_MEMBERS_0 = new StringIdImpl(1137, "Failed to send old value to the following members: {0}");
 +  public static final StringId BucketAdvisor_ATTEMPTED_TO_CLOSE_BUCKETADVISOR_THAT_IS_ALREADY_CLOSED = new StringId(1131, "Attempted to close BucketAdvisor that is already CLOSED");
 +  public static final StringId AgentImpl_COULD_NOT_TAIL_0_BECAUSE_1 = new StringId(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 StringId(1133, "Used to specify a host name or IP address to give to clients so they can connect to a locator.");
 +  public static final StringId BucketAdvisor_BUCKETADVISOR_WAS_NOT_CLOSED_PROPERLY = new StringId(1134, "BucketAdvisor was not closed properly.");
 +  public static final StringId BucketBackupMessage_BUCKETBACKUPMESSAGE_DATA_STORE_NOT_CONFIGURED_FOR_THIS_MEMBER = new StringId(1135, "BucketBackupMessage: data store not configured for this member");
 +  public static final StringId BucketCleanupMessage_BUCKETCLEANUPMESSAGE_DATA_STORE_NOT_CONFIGURED_FOR_THIS_MEMBER = new StringId(1136, "BucketCleanupMessage: data store not configured for this member");
 +  public static final StringId BucketRegion_FAILED_TO_SEND_OLD_VALUE_TO_THE_FOLLOWING_MEMBERS_0 = new StringId(1137, "Failed to send old value to the following members: {0}");
    // ok to reuse 1138
 -  public static final StringId CFactory_COULD_NOT_CREATE_CF5IMPL = new StringIdImpl(1139, "Could not create CF5Impl");
 -  public static final StringId CacheClientNotifier_0_REGISTERCLIENT_EXCEPTION_ENCOUNTERED_IN_REGISTRATION_1 = new StringIdImpl(1140, "{0} :registerClient: Exception encountered in registration {1}");
 -  public static final StringId CacheClientNotifier_CACHECLIENTNOTIFIER_KEEPING_PROXY_FOR_DURABLE_CLIENT_NAMED_0_FOR_1_SECONDS_2 = new StringIdImpl(1141, "CacheClientNotifier: Keeping proxy for durable client named {0} for {1} seconds {2}.");
 -  public static final StringId CacheClientNotifier_CACHECLIENTNOTIFIER_THE_REQUESTED_DURABLE_CLIENT_HAS_THE_SAME_IDENTIFIER__0__AS_AN_EXISTING_DURABLE_CLIENT__1__DUPLICATE_DURABLE_CLIENTS_ARE_NOT_ALLOWED = new StringIdImpl(1142, "CacheClientNotifier: The requested durable client has the same identifier ( {0} ) as an existing durable client ( {1} ). Duplicate durable clients are not allowed.");
 -  public static final StringId CacheClientNotifier_CACHECLIENTNOTIFIER_UNSUCCESSFULLY_REGISTERED_CLIENT_WITH_IDENTIFIER__0 = new StringIdImpl(1143, "CacheClientNotifier: Unsuccessfully registered client with identifier  {0}");
 -  public static final StringId CacheClientNotifier_CANNOT_NOTIFY_CLIENTS_TO_PERFORM_OPERATION_0_ON_EVENT_1 = new StringIdImpl(1144, "CacheClientNotifier: Cannot notify clients to perform operation {0} on event {1}");
 -  public static final StringId CacheClientNotifier_EXCEPTION_OCCURRED_WHILE_PROCESSING_CQS = new StringIdImpl(1145, "Exception occurred while processing CQs");
 -  public static final StringId CacheClientNotifier_NOTIFYING_CLIENTS_BY_SUBSCRIPTION_0 = new StringIdImpl(1146, "Notifying clients by subscription: {0}");
 -  public static final StringId CacheClientNotifier_UNABLE_TO_CLOSE_CQS_FOR_THE_CLIENT__0 = new StringIdImpl(1147, "Unable to close CQs for the client:  {0}");
 -  public static final StringId CacheClientNotifier_UNABLE_TO_GET_THE_CQSERVICE_WHILE_CLOSING_THE_DEAD_PROXIES = new StringIdImpl(1148, "Unable to get the CqService while closing the dead proxies");
 -  public static final StringId CacheClientProxy_0_AN_UNEXPECTED_IOEXCEPTION_OCCURRED_SO_THE_PROXY_WILL_BE_CLOSED = new StringIdImpl(1149, "{0}: An unexpected IOException occurred so the proxy will be closed.");
 -  public static final StringId CacheClientProxy_0_CANCELLING_EXPIRATION_TASK_SINCE_THE_CLIENT_HAS_RECONNECTED = new StringIdImpl(1150, "{0}: Cancelling expiration task since the client has reconnected.");
 -  public static final StringId CacheClientProxy_0_COULD_NOT_STOP_MESSAGE_DISPATCHER_THREAD = new StringIdImpl(1151, "{0}: Could not stop message dispatcher thread.");
 -  public static final StringId CacheClientProxy_0_EXCEPTION_IN_CLOSING_THE_UNDERLYING_HAREGION_OF_THE_HAREGIONQUEUE = new StringIdImpl(1152, "{0}: Exception in closing the underlying HARegion of the HARegionQueue");
 -  public static final StringId CacheClientProxy_0_EXCEPTION_OCCURRED_WHILE_ATTEMPTING_TO_ADD_MESSAGE_TO_QUEUE = new StringIdImpl(1153, "{0}: Exception occurred while attempting to add message to queue");
 -  public static final StringId CacheClientProxy_0_POSSIBILITY_OF_NOT_BEING_ABLE_TO_SEND_SOME_OR_ALL_THE_MESSAGES_TO_CLIENTS_TOTAL_MESSAGES_CURRENTLY_PRESENT_IN_THE_LIST_1 = new StringIdImpl(1154, "{0} Possibility of not being able to send some or all of the messages to clients. Total messages currently present in the list {1}.");
 -  public static final StringId CacheClientProxy_0_PROXY_CLOSING_DUE_TO_SOCKET_BEING_CLOSED_LOCALLY = new StringIdImpl(1155, "{0}: Proxy closing due to socket being closed locally.");
 -  public static final StringId CacheClientProxy_0_PROXY_CLOSING_DUE_TO_UNEXPECTED_BROKEN_PIPE_ON_SOCKET_CONNECTION = new StringIdImpl(1156, "{0}: Proxy closing due to unexpected broken pipe on socket connection.");
 -  public static final StringId CacheClientProxy_0_PROXY_CLOSING_DUE_TO_UNEXPECTED_RESET_BY_PEER_ON_SOCKET_CONNECTION = new StringIdImpl(1157, "{0}: Proxy closing due to unexpected reset by peer on socket connection.");
 -  public static final StringId CacheClientProxy_0_PROXY_CLOSING_DUE_TO_UNEXPECTED_RESET_ON_SOCKET_CONNECTION = new StringIdImpl(1158, "{0}: Proxy closing due to unexpected reset on socket connection.");
 -  public static final StringId CacheClientProxy_0__AN_UNEXPECTED_EXCEPTION_OCCURRED = new StringIdImpl(1159, "{0} : An unexpected Exception occurred");
 -  public static final StringId CacheClientProxy_0__EXCEPTION_OCCURRED_WHILE_ATTEMPTING_TO_ADD_MESSAGE_TO_QUEUE = new StringIdImpl(1160, "{0} : Exception occurred while attempting to add message to queue");
 -  public static final StringId CacheClientProxy_0__PAUSING_PROCESSING = new StringIdImpl(1161, "{0} : Pausing processing");
 -  public static final StringId CacheClientProxy_0__RESUMING_PROCESSING = new StringIdImpl(1162, "{0} : Resuming processing");
 -  public static final StringId CacheClientProxy_0__THE_EXPIRATION_TASK_HAS_FIRED_SO_THIS_PROXY_IS_BEING_TERMINATED = new StringIdImpl(1163, "{0} : The expiration task has fired, so this proxy is being terminated.");
 -  public static final StringId CacheClientProxy_CONNECTIONPROXYIMPL_COULD_NOT_RECOVER_INSTANTIATORS_WITH_0 = new StringIdImpl(1164, "ConnectionProxyImpl: Could not recover instantiators with {0}.");
 -  public static final StringId CacheClientProxy_CONNECTIONPROXYIMPL_COULD_NOT_RECOVER_INSTANTIATORS_WITH_0_BECAUSE_1 = new StringIdImpl(1165, "ConnectionProxyImpl: Could not recover instantiators with {0} because {1}");
 -  public static final StringId CacheClientProxy_PROBLEM_CAUSED_BY_BROKEN_PIPE_ON_SOCKET = new StringIdImpl(1166, "Problem caused by broken pipe on socket.");
 -  public static final StringId CacheClientProxy_PROBLEM_CAUSED_BY_MESSAGE_QUEUE_BEING_CLOSED = new StringIdImpl(1167, "Problem caused by message queue being closed.");
 -  public static final StringId CacheClientUpdater_0_CAUGHT_FOLLOWING_EXECPTION_WHILE_ATTEMPTING_TO_CREATE_A_SERVER_TO_CLIENT_COMMUNICATION_SOCKET_AND_WILL_EXIT_1 = new StringIdImpl(1168, "{0}: Caught following exception while attempting to create a server-to-client communication socket and will exit: {1}");
 -  public static final StringId CacheClientUpdater_0_CONNECTION_WAS_REFUSED = new StringIdImpl(1169, "{0} connection was refused");
 -  public static final StringId CacheClientUpdater_SSL_NEGOTIATION_FAILED_WITH_ENDPOINT_0 = new StringIdImpl(1170, "SSL negotiation failed with endpoint: {0}");
 -  public static final StringId CacheClientUpdater_0_EXITING_WITHOUT_READING_SERVER_MESSAGES_BECAUSE_OF_1_REGISTERINTEREST_CALLS_BEING_IN_PROGRESS = new StringIdImpl(1171, "{0}: exiting without reading server messages because of {1} registerInterest calls being in progress.");
 -  public static final StringId CacheClientUpdater_0_RECEIVED_AN_UNSUPPORTED_MESSAGE_TYPE_1 = new StringIdImpl(1172, "{0}: Received an unsupported message (type={1})");
 -  public static final StringId CacheClientUpdater_0__1__2 = new StringIdImpl(1173, "{0} :  {1} : {2}");
 -  public static final StringId CacheClientUpdater_THE_FOLLOWING_EXCEPTION_OCCURRED_WHILE_ATTEMPTING_TO_DESTROY_ENTRY_REGION_0_KEY_1 = new StringIdImpl(1174, "The following exception occurred while attempting to destroy entry (region: {0} key: {1})");
 -  public static final StringId CacheClientUpdater_CACHE_UPDATE_THREAD_FINISHED_WAITING_FOR_REGISTERINTEREST_TO_COMPLETE = new StringIdImpl(1175, "Cache update thread finished waiting for registerInterest to complete");
 -  public static final StringId CacheClientUpdater_CACHE_UPDATE_THREAD_WAITED_MORE_THAN_0_SEC_FOR_REGISTERINTEREST_TO_COMPLETE = new StringIdImpl(1176, "Cache update thread waited more than {0} sec for registerInterest to complete");
 -  public static final StringId CacheClientUpdater_FAILED_TO_INVOKE_CQ_DISPATCHER_ERROR___0 = new StringIdImpl(1177, "Failed to invoke CQ Dispatcher. Error :  {0}");
 -  public static final StringId CacheDistributionAdvisor_WAITING_FOR_CURRENT_OPERATIONS_TO_FINISH_0_ = new StringIdImpl(1178, "Waiting for current operations to finish( {0} )");
 -  public static final StringId CacheFactory_0_AN_OPEN_CACHE_ALREADY_EXISTS = new StringIdImpl(1179, "{0}: An open cache already exists.");
 -  public static final StringId InternalDistributedSystem_shutdownHook_shuttingdown = new StringIdImpl(1180, "VM is exiting - shutting down distributed system");
 -  public static final StringId JGroupMembershipManager_entered_into_membership_in_group_0_with_id_1 = new StringIdImpl(1181, "Entered into membership with ID {0}.");
 -  public static final StringId CacheServerLauncher_CACHE_SERVER_ERROR = new StringIdImpl(1182, "Cache server error");
 -  public static final StringId CacheXmlParser_XML_PARSER_CHARACTERS_APPENDED_CHARACTER_DATA_0 = new StringIdImpl(1183, "XML Parser characters, appended character data: {0}");
 -  public static final StringId CacheXmlParser_XML_PARSER_CHARACTERS_NEW_CHARACTER_DATA_0 = new StringIdImpl(1184, "XML Parser characters, new character data: {0}");
 -  public static final StringId CacheXmlParser_XML_PARSER_CREATEDECLARABLE_CLASS_NAME_0 = new StringIdImpl(1185, "XML Parser createDeclarable class name: {0}");
 -  public static final StringId CacheXmlParser_XML_PARSER_CREATEDECLARABLE_PROPERTIES__0 = new StringIdImpl(1186, "XML Parser createDeclarable properties:  {0}");
 -  public static final StringId ClearRegion_0_THE_INPUT_REGION_NAME_FOR_THE_CLEAR_REGION_REQUEST_IS_NULL = new StringIdImpl(1187, "{0}: The input region name for the clear region request is null");
 -  public static final StringId ClearRegion_THE_INPUT_REGION_NAME_FOR_THE_CLEAR_REGION_REQUEST_IS_NULL = new StringIdImpl(1188, " The input region name for the clear region request is null");
 -  public static final StringId ClearRegion_WAS_NOT_FOUND_DURING_CLEAR_REGION_REGUEST = new StringIdImpl(1189, " was not found during clear region request");
 -  public static final StringId ClientHealtMonitor_0_IS_BEING_TERMINATED_BECAUSE_ITS_CLIENT_TIMEOUT_OF_1_HAS_EXPIRED = new StringIdImpl(1190, "{0} is being terminated because its client timeout of {1} has expired.");
 -  public static final StringId ClientHealthMonitor_0_AN_UNEXPECTED_EXCEPTION_OCCURRED = new StringIdImpl(1191, "{0}: An unexpected Exception occurred");
 -  public static final StringId ClientHealthMonitor_CLIENTHEALTHMONITORTHREAD_MAXIMUM_ALLOWED_TIME_BETWEEN_PINGS_0 = new StringIdImpl(1192, "ClientHealthMonitorThread maximum allowed time between pings: {0}");
 -  public static final StringId ClientHealthMonitor_CLIENTHEALTHMONITOR_REGISTERING_CLIENT_WITH_MEMBER_ID_0 = new StringIdImpl(1193, "ClientHealthMonitor: Registering client with member id {0}");
 -  public static final StringId ClientHealthMonitor_CLIENTHEALTHMONITOR_UNREGISTERING_CLIENT_WITH_MEMBER_ID_0 = new StringIdImpl(1194, "ClientHealthMonitor: Unregistering client with member id {0}");
 -  public static final StringId ClientHealthMonitor_CLIENT_HEALTH_MONITOR_THREAD_DISABLED_DUE_TO_MAXIMUMTIMEBETWEENPINGS_SETTING__0 = new StringIdImpl(1195, "Client health monitor thread disabled due to maximumTimeBetweenPings setting:  {0}");
 -  public static final StringId ClientHealthMonitor_MONITORING_CLIENT_WITH_MEMBER_ID_0_IT_HAD_BEEN_1_MS_SINCE_THE_LATEST_HEARTBEAT_MAX_INTERVAL_IS_2_TERMINATED_CLIENT = new StringIdImpl(1196, "Monitoring client with member id {0}. It had been {1} ms since the latest heartbeat. Max interval is {2}. Terminated client.");
 -  public static final StringId ClientHealthMonitor_UNEXPECTED_INTERRUPT_EXITING = new StringIdImpl(1197, "Unexpected interrupt, exiting");
 -  public static final StringId ClientProxyMembershipID_UNABLE_TO_DESERIALIZE_MEMBERSHIP_ID = new StringIdImpl(1198, "Unable to deserialize membership id");
 -  public static final StringId DiskStore_IS_USED_IN_NONPERSISTENT_REGION = new StringIdImpl(1199, "Only regions with persistence or overflow to disk can specify DiskStore");
 -  public static final StringId DiskRegion_COMPLEXDISKREGIONGETNEXTDIR_MAX_DIRECTORY_SIZE_WILL_GET_VIOLATED__GOING_AHEAD_WITH_THE_SWITCHING_OF_OPLOG_ANY_WAYS_CURRENTLY_AVAILABLE_SPACE_IN_THE_DIRECTORY_IS__0__THE_CAPACITY_OF_DIRECTORY_IS___1 = new StringIdImpl(1200, "Even though the configured directory size limit has been exceeded a new oplog will be created because compaction is enabled. The configured limit is {1}. The current space used in the directory by this disk store is {0}.");
 -  public static final StringId DiskRegion_PROBLEM_IN_COMPACTOR_THREAD_IT_WILL_TERMINATE = new StringIdImpl(1201, "Problem in Compactor thread. It will terminate.");
 -  public static final StringId DiskRegion_THE_ENTRY_IN_QUESTION_IS_HAVING_DISKID_AS_0 = new StringIdImpl(1202, "The entry in question is having DiskId = {0}");
 -  public static final StringId ConnectionImpl_0_INTERRUPTED_READING_PING_RESPONSE_FOR_TRANSACTION_1 = new StringIdImpl(1203, "{0}: Interrupted reading ping response for transaction {1}");
 -  public static final StringId ConnectionImpl_0_SOCKETTIMEOUTEXCEPTION_DURING_BATCH_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringIdImpl(1204, "{0}: SocketTimeoutException during batch operation on region: {1} key: {2} messageId: {3}");
 -  public static final StringId ConnectionImpl_0_SOCKETTIMEOUTEXCEPTION_DURING_CQ_OPERATION = new StringIdImpl(1205, "{0}: SocketTimeoutException during cq operation.");
 -  public static final StringId ConnectionImpl_0_SOCKETTIMEOUTEXCEPTION_DURING_DESTROY_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringIdImpl(1206, "{0}: SocketTimeoutException during destroy operation on region: {1} key: {2} messageId: {3}");
 -  public static final StringId ConnectionImpl_0_SOCKETTIMEOUTEXCEPTION_DURING_PUT_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringIdImpl(1207, "{0}: SocketTimeoutException during put operation on region: {1} key: {2} messageId: {3}");
 -  public static final StringId ConnectionImpl_0_TIMED_OUT_READING_PING_RESPONSE_FOR_TRANSACTION_1 = new StringIdImpl(1208, "{0}: Timed out reading ping response for transaction {1}");
 -  public static final StringId ConnectionImpl_AN_INCORRECT_CONNECTION_HAS_BEEN_ASSIGNED_THIS_CONNECTION_IS_ALREADY_PROCESSING_A_REQUEST = new StringIdImpl(1209, "An incorrect connection has been assigned. This connection is already processing a request.");
 -  public static final StringId ConnectionImpl_CONNECTION_0_WHILE_PERFORMING_A_REMOTE_CLIENTREADYTORECEIVEUPDATES = new StringIdImpl(1210, "Connection ({0}): While performing a remote clientReadyToReceiveUpdates");
 -  public static final StringId ConnectionImpl_THE_CACHESERVER_ON_0_HAS_DIED_UNEXPECTEDLY = new StringIdImpl(1211, "The CacheServer on {0} has died unexpectedly");
 -  public static final StringId ConnectionImpl_UNEXPECTED_MESSAGE_TYPE_0 = new StringIdImpl(1212, "Unexpected message type {0}");
 -  public static final StringId AttributesFactory_CLONENOTSUPPORTEDEXCEPTION_THROWN_IN_CLASS_THAT_IMPLEMENTS_CLONEABLE = new StringIdImpl(1213, "CloneNotSupportedException thrown in class that implements cloneable.");
 -  public static final StringId ConnectionProxyImpl_0_FAILED_TO_SET_PRIMARY_ENDPOINT_TO_1_2 = new StringIdImpl(1214, "{0}: Failed to set primary endpoint to <{1}>: {2}");
 -  public static final StringId CqQueryImpl_CQ_IS_CLOSED_CQNAME_0 = new StringIdImpl(1215, "CQ is closed, CqName : {0}");
 -  public static final StringId ConnectionProxyImpl_0__ADDING__1 = new StringIdImpl(1216, "{0} : Adding  {1}");
 -  public static final StringId ConnectionProxyImpl_0__ATTEMPTING_TO_REMOVE_ENDPOINT__1__2__3 = new StringIdImpl(1217, "{0} : Attempting to remove endpoint  {1} -> {2} : {3}");
 -  public static final StringId ConnectionProxyImpl_0__CONFIGURATION__1 = new StringIdImpl(1218, "{0} : configuration:  {1}");
 -  public static final StringId ConnectionProxyImpl_0__INITIALIZED_USING__1 = new StringIdImpl(1219, "{0} : initialized using:  {1}");
 -  public static final StringId ConnectionProxyImpl_0__LIVE__1__CANNOT_BE_REMOVED_BECAUSE_IT_HAS_CONNECTIONS = new StringIdImpl(1220, "{0} : Live  {1}  cannot be removed because it has connections.");
 -  public static final StringId ConnectionProxyImpl_0__LIVE__1__HAS_BEEN_REMOVED = new StringIdImpl(1221, "{0} : Live  {1}  has been removed.");
 -  public static final StringId ConnectionProxyImpl_0__NO_EXISTING_LIVE_ENDPOINT_WAS_FOUND_FOR_ENDPOINT__1__2__3__NO_ENDPOINT_WAS_REMOVED = new StringIdImpl(1222, "{0} : No existing live endpoint was found for endpoint  {1} -> {2} : {3} . No endpoint was removed.");
 -  public static final StringId ConnectionProxyImpl_0__SUCCESSFULLY_SENT_CLIENT_READY_MESSAGE_AFTER_FAILOVER_TO__1 = new StringIdImpl(1223, "{0} : Successfully sent client ready message after failover to  {1}");
 -  public static final StringId ConnectionProxyImpl_0__SUCCESSFULLY_SENT_CLIENT_READY_MESSAGE_TO__1 = new StringIdImpl(1224, "{0} : Successfully sent client ready message to  {1}");
 -  public static final StringId ConnectionProxyImpl_0__THE_EXISTING_LIVE_ENDPOINT_NAMED__1__COULD_NOT_BE_REMOVED_BECAUSE_ITS_HOST_AND_PORT__2__3__DOES_NOT_MATCH_THE_REQUESTED_HOST_AND_PORT__4__5_ = new StringIdImpl(1225, "{0} : The existing live endpoint named  {1}  could not be removed because its host and port ( {2} : {3} ) does not match the requested host and port ( {4} : {5} ).");
 -  public static final StringId Connection_DISCONNECTED_AS_A_SLOWRECEIVER = new StringIdImpl(1226, "Disconnected as a slow-receiver");
 -  public static final StringId ConnectionProxyImpl_CACHE_IS_CLOSED = new StringIdImpl(1227, "Cache is closed");
 -  public static final StringId ConnectionProxyImpl_CONDITIONALLYRELEASECONNECTION_FAILED_ON_0_DUE_TO_UNKNOWN_POLICY_TYPE_1 = new StringIdImpl(1228, "conditionallyReleaseConnection: failed on <{0}> due to unknown policy type {1}");
 -  public static final StringId ConnectionProxyImpl_CONNECTIONPROXYIMPL_0_FINISHED_INSTANTIATORS_RECOVERY = new StringIdImpl(1229, "ConnectionProxyImpl ({0}) finished instantiators recovery.");
 -  public static final StringId ConnectionProxyImpl_CONNECTIONPROXYIMPL_0_STARTED_INSTANTIATORS_RECOVERY = new StringIdImpl(1230, "ConnectionProxyImpl ({0}) started instantiators recovery.");
 -  public static final StringId ConnectionProxyImpl_CONNECTIONPROXYIMPL_COULD_NOT_RECOVER_INSTANTIATORS_WITH_0_1 = new StringIdImpl(1231, "ConnectionProxyImpl: Could not recover instantiators with {0}. {1}");
 -  public static final StringId ConnectionProxyImpl_COULD_NOT_ACQUIRE_OR_CREATE_A_CONNECTION_NO_ACTIVE_SERVERS_WERE_FOUND_WHILE_REGISTERING_INSTANTIATORS = new StringIdImpl(1232, "Could not acquire or create a Connection. No active servers were found while registering instantiators.");
 -  public static final StringId ConnectionProxyImpl_COULD_NOT_ACQUIRE_OR_CREATE_A_CONNECTION_NO_AVAILABLE_CONNECTION_WAS_FOUND_WHILE_REGISTERING_INSTANTIATORS_BUT_THE_FOLLOWING_ACTIVE_SERVERS_EXIST_0 = new StringIdImpl(1233, "Could not acquire or create a Connection. No available Connection was found while registering instantiators, but the following active servers exist: {0}");
 -  public static final StringId ConnectionProxyImpl_ERROR_WHILE_REREGISTERING_THE_CQ_TO_THE_REDUNDANT_SERVER_CQNAME_0_ERROR_1 = new StringIdImpl(1234, "Error while re-registering the CQ to the redundant server. CqName : {0}, Error : {1}");
 -  public static final StringId ConnectionProxyImpl_EXCEPTION_WHILE_SENDING_AN_ACK_TO_THE_PRIMARY_SERVER = new StringIdImpl(1235, "Exception while sending an ack to the primary server");
 -  public static final StringId ConnectionProxyImpl_PROBLEM_REMOVING_ALL_INTEREST_ON_REGION_0_INTERESTTYPE_1 = new StringIdImpl(1236, "Problem removing all interest on region={0} interestType={1}");
 -  public static final StringId ConnectionProxyImpl_PROXY_NOT_PROPERLY_INITIALIZED_0 = new StringIdImpl(1237, "Proxy not properly initialized: {0}");
 -  public static final StringId ConnectionProxyImpl_THREADIDTOSEQUENCEIDEXPIRYTHREADUNEXPECTED_INTRREUPT_HENCE_CONTINUE_RUNNING = new StringIdImpl(1238, "ThreadIdToSequenceIdExpiryThread::Unexpected interrupt, hence continue running");
 -  public static final StringId ConnectionProxyImpl_UNEXPECTED_EXCEPTION_OCCURRED_WHILE_ATTEMPTING_TO_NOTIFY_PRIMARY_SERVER__0__OF_CLIENT_READINESS = new StringIdImpl(1239, "Unexpected exception occurred while attempting to notify primary server  {0}  of client readiness");
 -  public static final StringId ConnectionTable_FAILED_TO_ACCEPT_CONNECTION_FROM_0_BECAUSE_1 = new StringIdImpl(1240, "Failed to accept connection from {0} because: {1}");
 -  public static final StringId ConnectionTable_KEY_0___VALUE_HASH_1__DESCR_2 = new StringIdImpl(1241, "key= {0}   value hash= {1}  descr= {2}");
 -  public static final StringId ConnectionTable_P2P_CONNECTION_TABLE_CONTENTS = new StringIdImpl(1242, "P2P connection table contents-----------");
 -  public static final StringId Connection_0_ASYNC_CONFIGURATION_RECEIVED_1 = new StringIdImpl(1243, "{0} async configuration received {1}.");
 -  public static final StringId Connection_0_ERROR_READING_MESSAGE = new StringIdImpl(1244, "{0} Error reading message");
 -  public static final StringId Connection_0_EXCEPTION_IN_CHANNEL_READ = new StringIdImpl(1245, "{0} exception in channel read");
 -  public static final StringId Connection_0_EXCEPTION_RECEIVED = new StringIdImpl(1246, "{0} exception received");
 -  public static final StringId Connection_0_STRAY_INTERRUPT_READING_MESSAGE = new StringIdImpl(1247, "{0} Stray interrupt reading message");
 -  public static final StringId Connection_0_SUCCESSFULLY_REESTABLISHED_CONNECTION_TO_PEER_1 = new StringIdImpl(1248, "{0}: Successfully reestablished connection to peer {1}");
 -  public static final StringId Connection_ACK_READ_EXCEPTION = new StringIdImpl(1249, "ack read exception");
 -  public static final StringId Connection_ACK_READ_EXCEPTION_0 = new StringIdImpl(1250, "ack read exception: {0}");
 -  public static final StringId Connection_ACK_WAIT_THRESHOLD_EXCEEDED_WAITING_FOR_ACK_FROM_0 = new StringIdImpl(1251, "Ack wait threshold exceeded waiting for ack from {0}");
 -  public static final StringId Connection_ACK_WRITE_EXCEPTION = new StringIdImpl(1252, "ack write exception");
 -  public static final StringId Connection_ACK_WRITE_EXCEPTION_0 = new StringIdImpl(1253, "ack write exception: {0}");
 -  public static final StringId Connection_ALLOCATING_LARGER_NETWORK_READ_BUFFER_NEW_SIZE_IS_0_OLD_SIZE_WAS_1 = new StringIdImpl(1254, "Allocating larger network read buffer, new size is {0} old size was {1}.");
 -  public static final StringId Connection_BLOCKED_FOR_0_MS_WHICH_IS_LONGER_THAN_THE_MAX_OF_1_MS_ASKING_SLOW_RECEIVER_2_TO_DISCONNECT = new StringIdImpl(1255, "Blocked for {0}ms which is longer than the max of {1}ms, asking slow receiver {2} to disconnect.");
 -  public static final StringId Connection_CLASSNOTFOUND_DESERIALIZING_MESSAGE_0 = new StringIdImpl(1256, "ClassNotFound deserializing message: {0}");
 -  public static final StringId Connection_CONNECTION_ATTEMPTING_RECONNECT_TO_PEER__0 = new StringIdImpl(1257, "Connection: Attempting reconnect to peer  {0}");
 -  public static final StringId Connection_CONNECTION_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1 = new StringIdImpl(1258, "Connection: shared={0} ordered={1} failed to connect to peer {2} because: {3}");
 -  public static final StringId Connection_CONNECTION_HANDSHAKE_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1 = new StringIdImpl(1259, "Connection: shared={0} ordered={1} handshake failed to connect to peer {2} because: {3}");
 -  public static final StringId Connection_DETECTED_OLD_VERSION_PRE_5_0_1_OF_GEMFIRE_OR_NONGEMFIRE_DURING_HANDSHAKE_DUE_TO_INITIAL_BYTE_BEING_0 = new StringIdImpl(1260, "Detected old version (pre 5.0.1) of GemFire or non-GemFire during handshake due to initial byte being {0}");
 -  public static final StringId Connection_END_OF_FILE_ON_ACK_STREAM = new StringIdImpl(1261, "end of file on ack stream");
 -  public static final StringId Connection_ERROR_DESERIALIZING_MESSAGE = new StringIdImpl(1262, "Error deserializing message");
 -  public static final StringId Connection_ERROR_DESERIALIZING_P2P_HANDSHAKE_MESSAGE = new StringIdImpl(1263, "Error deserializing P2P handshake message");
 -  public static final StringId Connection_ERROR_DESERIALIZING_P2P_HANDSHAKE_REPLY = new StringIdImpl(1264, "Error deserializing P2P handshake reply");
 -  public static final StringId Connection_ERROR_DISPATCHING_MESSAGE = new StringIdImpl(1265, "Error dispatching message");
 -  public static final StringId Connection_EXCEPTION_FLUSHING_BATCH_SEND_BUFFER_0 = new StringIdImpl(1266, "Exception flushing batch send buffer: {0}");
 -  public static final StringId Connection_FAILED_HANDLING_CHUNK_MESSAGE = new StringIdImpl(1267, "Failed handling chunk message");
 -  public static final StringId Connection_FAILED_HANDLING_END_CHUNK_MESSAGE = new StringIdImpl(1268, "Failed handling end chunk message");
 -  public static final StringId Connection_FAILED_SETTING_CHANNEL_TO_BLOCKING_MODE_0 = new StringIdImpl(1269, "Failed setting channel to blocking mode {0}");
 -  public static final StringId Connection_FINISHED_WAITING_FOR_REPLY_FROM_0 = new StringIdImpl(1270, "Finished waiting for reply from {0}");
 -  public static final StringId Connection_IOEXCEPTION_DESERIALIZING_MESSAGE = new StringIdImpl(1271, "IOException deserializing message");
 -  public static final StringId Connection_OWNER_SHOULD_NOT_BE_NULL = new StringIdImpl(1272, "\"owner\" should not be null");
 -  public static final StringId Connection_P2P_PUSHER_EXCEPTION_0 = new StringIdImpl(1273, "P2P pusher exception: {0}");
 -  public static final StringId Connection_QUEUED_BYTES_0_EXCEEDS_MAX_OF_1_ASKING_SLOW_RECEIVER_2_TO_DISCONNECT = new StringIdImpl(1274, "Queued bytes {0} exceeds max of {1}, asking slow receiver {2} to disconnect.");
 -  public static final StringId Connection_SOCKET_0_IS_1_INSTEAD_OF_THE_REQUESTED_2 = new StringIdImpl(1275, "Socket {0} is {1} instead of the requested {2}.");
 -  public static final StringId Connection_THROWABLE_DESERIALIZING_P2P_HANDSHAKE_REPLY = new StringIdImpl(1276, "Throwable deserializing P2P handshake reply");
 -  public static final StringId Connection_THROWABLE_DISPATCHING_MESSAGE = new StringIdImpl(1277, "Throwable dispatching message");
 -  public static final StringId Connection_TIMED_OUT_WAITING_FOR_READERTHREAD_ON_0_TO_FINISH = new StringIdImpl(1278, "Timed out waiting for readerThread on {0} to finish.");
 -  public static final StringId Connection_UNABLE_TO_GET_INPUT_STREAM = new StringIdImpl(1279, "Unable to get input stream");
 -  public static final StringId Connection_UNABLE_TO_GET_P2P_CONNECTION_STREAMS = new StringIdImpl(1280, "Unable to get P2P connection streams");
 -  public static final StringId Connection_UNEXPECTED_FAILURE_DESERIALIZING_MESSAGE = new StringIdImpl(1281, "Unexpected failure deserializing message");
 -  public static final StringId Connection_UNKNOWN_HANDSHAKE_REPLY_CODE_0 = new StringIdImpl(1282, "Unknown handshake reply code: {0}");
 -  public static final StringId Connection_UNKNOWN_HANDSHAKE_REPLY_CODE_0_NIOMESSAGELENGTH_1_PROCESSORTYPE_2 = new StringIdImpl(1283, "Unknown handshake reply code: {0} nioMessageLength={1} processorType={2}");
 -  public static final StringId Connection_UNKNOWN_P2P_MESSAGE_TYPE_0 = new StringIdImpl(1284, "Unknown P2P message type: {0}");
 -  public static final StringId Connection_UNKNOWN_PROCESSOR_TYPE_0 = new StringIdImpl(1285, "Unknown processor type: {0}");
 -  public static final StringId ContainsKeyValueMess_PARTITIONED_REGION_0_IS_NOT_CONFIGURED_TO_STORE_DATA = new StringIdImpl(1286, "Partitioned Region {0} is not configured to store data");
 +  public static final StringId CFactory_COULD_NOT_CREATE_CF5IMPL = new StringId(1139, "Could not create CF5Impl");
 +  public static final StringId CacheClientNotifier_0_REGISTERCLIENT_EXCEPTION_ENCOUNTERED_IN_REGISTRATION_1 = new StringId(1140, "{0} :registerClient: Exception encountered in registration {1}");
 +  public static final StringId CacheClientNotifier_CACHECLIENTNOTIFIER_KEEPING_PROXY_FOR_DURABLE_CLIENT_NAMED_0_FOR_1_SECONDS_2 = new StringId(1141, "CacheClientNotifier: Keeping proxy for durable client named {0} for {1} seconds {2}.");
 +  public static final StringId CacheClientNotifier_CACHECLIENTNOTIFIER_THE_REQUESTED_DURABLE_CLIENT_HAS_THE_SAME_IDENTIFIER__0__AS_AN_EXISTING_DURABLE_CLIENT__1__DUPLICATE_DURABLE_CLIENTS_ARE_NOT_ALLOWED = new StringId(1142, "CacheClientNotifier: The requested durable client has the same identifier ( {0} ) as an existing durable client ( {1} ). Duplicate durable clients are not allowed.");
 +  public static final StringId CacheClientNotifier_CACHECLIENTNOTIFIER_UNSUCCESSFULLY_REGISTERED_CLIENT_WITH_IDENTIFIER__0 = new StringId(1143, "CacheClientNotifier: Unsuccessfully registered client with identifier  {0}");
 +  public static final StringId CacheClientNotifier_CANNOT_NOTIFY_CLIENTS_TO_PERFORM_OPERATION_0_ON_EVENT_1 = new StringId(1144, "CacheClientNotifier: Cannot notify clients to perform operation {0} on event {1}");
 +  public static final StringId CacheClientNotifier_EXCEPTION_OCCURRED_WHILE_PROCESSING_CQS = new StringId(1145, "Exception occurred while processing CQs");
 +  public static final StringId CacheClientNotifier_NOTIFYING_CLIENTS_BY_SUBSCRIPTION_0 = new StringId(1146, "Notifying clients by subscription: {0}");
 +  public static final StringId CacheClientNotifier_UNABLE_TO_CLOSE_CQS_FOR_THE_CLIENT__0 = new StringId(1147, "Unable to close CQs for the client:  {0}");
 +  public static final StringId CacheClientNotifier_UNABLE_TO_GET_THE_CQSERVICE_WHILE_CLOSING_THE_DEAD_PROXIES = new StringId(1148, "Unable to get the CqService while closing the dead proxies");
 +  public static final StringId CacheClientProxy_0_AN_UNEXPECTED_IOEXCEPTION_OCCURRED_SO_THE_PROXY_WILL_BE_CLOSED = new StringId(1149, "{0}: An unexpected IOException occurred so the proxy will be closed.");
 +  public static final StringId CacheClientProxy_0_CANCELLING_EXPIRATION_TASK_SINCE_THE_CLIENT_HAS_RECONNECTED = new StringId(1150, "{0}: Cancelling expiration task since the client has reconnected.");
 +  public static final StringId CacheClientProxy_0_COULD_NOT_STOP_MESSAGE_DISPATCHER_THREAD = new StringId(1151, "{0}: Could not stop message dispatcher thread.");
 +  public static final StringId CacheClientProxy_0_EXCEPTION_IN_CLOSING_THE_UNDERLYING_HAREGION_OF_THE_HAREGIONQUEUE = new StringId(1152, "{0}: Exception in closing the underlying HARegion of the HARegionQueue");
 +  public static final StringId CacheClientProxy_0_EXCEPTION_OCCURRED_WHILE_ATTEMPTING_TO_ADD_MESSAGE_TO_QUEUE = new StringId(1153, "{0}: Exception occurred while attempting to add message to queue");
 +  public static final StringId CacheClientProxy_0_POSSIBILITY_OF_NOT_BEING_ABLE_TO_SEND_SOME_OR_ALL_THE_MESSAGES_TO_CLIENTS_TOTAL_MESSAGES_CURRENTLY_PRESENT_IN_THE_LIST_1 = new StringId(1154, "{0} Possibility of not being able to send some or all of the messages to clients. Total messages currently present in the list {1}.");
 +  public static final StringId CacheClientProxy_0_PROXY_CLOSING_DUE_TO_SOCKET_BEING_CLOSED_LOCALLY = new StringId(1155, "{0}: Proxy closing due to socket being closed locally.");
 +  public static final StringId CacheClientProxy_0_PROXY_CLOSING_DUE_TO_UNEXPECTED_BROKEN_PIPE_ON_SOCKET_CONNECTION = new StringId(1156, "{0}: Proxy closing due to unexpected broken pipe on socket connection.");
 +  public static final StringId CacheClientProxy_0_PROXY_CLOSING_DUE_TO_UNEXPECTED_RESET_BY_PEER_ON_SOCKET_CONNECTION = new StringId(1157, "{0}: Proxy closing due to unexpected reset by peer on socket connection.");
 +  public static final StringId CacheClientProxy_0_PROXY_CLOSING_DUE_TO_UNEXPECTED_RESET_ON_SOCKET_CONNECTION = new StringId(1158, "{0}: Proxy closing due to unexpected reset on socket connection.");
 +  public static final StringId CacheClientProxy_0__AN_UNEXPECTED_EXCEPTION_OCCURRED = new StringId(1159, "{0} : An unexpected Exception occurred");
 +  public static final StringId CacheClientProxy_0__EXCEPTION_OCCURRED_WHILE_ATTEMPTING_TO_ADD_MESSAGE_TO_QUEUE = new StringId(1160, "{0} : Exception occurred while attempting to add message to queue");
 +  public static final StringId CacheClientProxy_0__PAUSING_PROCESSING = new StringId(1161, "{0} : Pausing processing");
 +  public static final StringId CacheClientProxy_0__RESUMING_PROCESSING = new StringId(1162, "{0} : Resuming processing");
 +  public static final StringId CacheClientProxy_0__THE_EXPIRATION_TASK_HAS_FIRED_SO_THIS_PROXY_IS_BEING_TERMINATED = new StringId(1163, "{0} : The expiration task has fired, so this proxy is being terminated.");
 +  public static final StringId CacheClientProxy_CONNECTIONPROXYIMPL_COULD_NOT_RECOVER_INSTANTIATORS_WITH_0 = new StringId(1164, "ConnectionProxyImpl: Could not recover instantiators with {0}.");
 +  public static final StringId CacheClientProxy_CONNECTIONPROXYIMPL_COULD_NOT_RECOVER_INSTANTIATORS_WITH_0_BECAUSE_1 = new StringId(1165, "ConnectionProxyImpl: Could not recover instantiators with {0} because {1}");
 +  public static final StringId CacheClientProxy_PROBLEM_CAUSED_BY_BROKEN_PIPE_ON_SOCKET = new StringId(1166, "Problem caused by broken pipe on socket.");
 +  public static final StringId CacheClientProxy_PROBLEM_CAUSED_BY_MESSAGE_QUEUE_BEING_CLOSED = new StringId(1167, "Problem caused by message queue being closed.");
 +  public static final StringId CacheClientUpdater_0_CAUGHT_FOLLOWING_EXECPTION_WHILE_ATTEMPTING_TO_CREATE_A_SERVER_TO_CLIENT_COMMUNICATION_SOCKET_AND_WILL_EXIT_1 = new StringId(1168, "{0}: Caught following exception while attempting to create a server-to-client communication socket and will exit: {1}");
 +  public static final StringId CacheClientUpdater_0_CONNECTION_WAS_REFUSED = new StringId(1169, "{0} connection was refused");
 +  public static final StringId CacheClientUpdater_SSL_NEGOTIATION_FAILED_WITH_ENDPOINT_0 = new StringId(1170, "SSL negotiation failed with endpoint: {0}");
 +  public static final StringId CacheClientUpdater_0_EXITING_WITHOUT_READING_SERVER_MESSAGES_BECAUSE_OF_1_REGISTERINTEREST_CALLS_BEING_IN_PROGRESS = new StringId(1171, "{0}: exiting without reading server messages because of {1} registerInterest calls being in progress.");
 +  public static final StringId CacheClientUpdater_0_RECEIVED_AN_UNSUPPORTED_MESSAGE_TYPE_1 = new StringId(1172, "{0}: Received an unsupported message (type={1})");
 +  public static final StringId CacheClientUpdater_0__1__2 = new StringId(1173, "{0} :  {1} : {2}");
 +  public static final StringId CacheClientUpdater_THE_FOLLOWING_EXCEPTION_OCCURRED_WHILE_ATTEMPTING_TO_DESTROY_ENTRY_REGION_0_KEY_1 = new StringId(1174, "The following exception occurred while attempting to destroy entry (region: {0} key: {1})");
 +  public static final StringId CacheClientUpdater_CACHE_UPDATE_THREAD_FINISHED_WAITING_FOR_REGISTERINTEREST_TO_COMPLETE = new StringId(1175, "Cache update thread finished waiting for registerInterest to complete");
 +  public static final StringId CacheClientUpdater_CACHE_UPDATE_THREAD_WAITED_MORE_THAN_0_SEC_FOR_REGISTERINTEREST_TO_COMPLETE = new StringId(1176, "Cache update thread waited more than {0} sec for registerInterest to complete");
 +  public static final StringId CacheClientUpdater_FAILED_TO_INVOKE_CQ_DISPATCHER_ERROR___0 = new StringId(1177, "Failed to invoke CQ Dispatcher. Error :  {0}");
 +  public static final StringId CacheDistributionAdvisor_WAITING_FOR_CURRENT_OPERATIONS_TO_FINISH_0_ = new StringId(1178, "Waiting for current operations to finish( {0} )");
 +  public static final StringId CacheFactory_0_AN_OPEN_CACHE_ALREADY_EXISTS = new StringId(1179, "{0}: An open cache already exists.");
 +  public static final StringId InternalDistributedSystem_shutdownHook_shuttingdown = new StringId(1180, "VM is exiting - shutting down distributed system");
 +  public static final StringId GroupMembershipService_entered_into_membership_in_group_0_with_id_1 = new StringId(1181, "Finished joining (took {0}ms).");
 +  public static final StringId CacheServerLauncher_CACHE_SERVER_ERROR = new StringId(1182, "Cache server error");
 +  public static final StringId CacheXmlParser_XML_PARSER_CHARACTERS_APPENDED_CHARACTER_DATA_0 = new StringId(1183, "XML Parser characters, appended character data: {0}");
 +  public static final StringId CacheXmlParser_XML_PARSER_CHARACTERS_NEW_CHARACTER_DATA_0 = new StringId(1184, "XML Parser characters, new character data: {0}");
 +  public static final StringId CacheXmlParser_XML_PARSER_CREATEDECLARABLE_CLASS_NAME_0 = new StringId(1185, "XML Parser createDeclarable class name: {0}");
 +  public static final StringId CacheXmlParser_XML_PARSER_CREATEDECLARABLE_PROPERTIES__0 = new StringId(1186, "XML Parser createDeclarable properties:  {0}");
 +  public static final StringId ClearRegion_0_THE_INPUT_REGION_NAME_FOR_THE_CLEAR_REGION_REQUEST_IS_NULL = new StringId(1187, "{0}: The input region name for the clear region request is null");
 +  public static final StringId ClearRegion_THE_INPUT_REGION_NAME_FOR_THE_CLEAR_REGION_REQUEST_IS_NULL = new StringId(1188, " The input region name for the clear region request is null");
 +  public static final StringId ClearRegion_WAS_NOT_FOUND_DURING_CLEAR_REGION_REGUEST = new StringId(1189, " was not found during clear region request");
 +  public static final StringId ClientHealtMonitor_0_IS_BEING_TERMINATED_BECAUSE_ITS_CLIENT_TIMEOUT_OF_1_HAS_EXPIRED = new StringId(1190, "{0} is being terminated because its client timeout of {1} has expired.");
 +  public static final StringId ClientHealthMonitor_0_AN_UNEXPECTED_EXCEPTION_OCCURRED = new StringId(1191, "{0}: An unexpected Exception occurred");
 +  public static final StringId ClientHealthMonitor_CLIENTHEALTHMONITORTHREAD_MAXIMUM_ALLOWED_TIME_BETWEEN_PINGS_0 = new StringId(1192, "ClientHealthMonitorThread maximum allowed time between pings: {0}");
 +  public static final StringId ClientHealthMonitor_CLIENTHEALTHMONITOR_REGISTERING_CLIENT_WITH_MEMBER_ID_0 = new StringId(1193, "ClientHealthMonitor: Registering client with member id {0}");
 +  public static final StringId ClientHealthMonitor_CLIENTHEALTHMONITOR_UNREGISTERING_CLIENT_WITH_MEMBER_ID_0 = new StringId(1194, "ClientHealthMonitor: Unregistering client with member id {0}");
 +  public static final StringId ClientHealthMonitor_CLIENT_HEALTH_MONITOR_THREAD_DISABLED_DUE_TO_MAXIMUMTIMEBETWEENPINGS_SETTING__0 = new StringId(1195, "Client health monitor thread disabled due to maximumTimeBetweenPings setting:  {0}");
 +  public static final StringId ClientHealthMonitor_MONITORING_CLIENT_WITH_MEMBER_ID_0_IT_HAD_BEEN_1_MS_SINCE_THE_LATEST_HEARTBEAT_MAX_INTERVAL_IS_2_TERMINATED_CLIENT = new StringId(1196, "Monitoring client with member id {0}. It had been {1} ms since the latest heartbeat. Max interval is {2}. Terminated client.");
 +  public static final StringId ClientHealthMonitor_UNEXPECTED_INTERRUPT_EXITING = new StringId(1197, "Unexpected interrupt, exiting");
 +  public static final StringId ClientProxyMembershipID_UNABLE_TO_DESERIALIZE_MEMBERSHIP_ID = new StringId(1198, "Unable to deserialize membership id");
 +  public static final StringId DiskStore_IS_USED_IN_NONPERSISTENT_REGION = new StringId(1199, "Only regions with persistence or overflow to disk can specify DiskStore");
 +  public static final StringId DiskRegion_COMPLEXDISKREGIONGETNEXTDIR_MAX_DIRECTORY_SIZE_WILL_GET_VIOLATED__GOING_AHEAD_WITH_THE_SWITCHING_OF_OPLOG_ANY_WAYS_CURRENTLY_AVAILABLE_SPACE_IN_THE_DIRECTORY_IS__0__THE_CAPACITY_OF_DIRECTORY_IS___1 = new StringId(1200, "Even though the configured directory size limit has been exceeded a new oplog will be created because compaction is enabled. The configured limit is {1}. The current space used in the directory by this disk store is {0}.");
 +  public static final StringId DiskRegion_PROBLEM_IN_COMPACTOR_THREAD_IT_WILL_TERMINATE = new StringId(1201, "Problem in Compactor thread. It will terminate.");
 +  public static final StringId DiskRegion_THE_ENTRY_IN_QUESTION_IS_HAVING_DISKID_AS_0 = new StringId(1202, "The entry in question is having DiskId = {0}");
 +  public static final StringId ConnectionImpl_0_INTERRUPTED_READING_PING_RESPONSE_FOR_TRANSACTION_1 = new StringId(1203, "{0}: Interrupted reading ping response for transaction {1}");
 +  public static final StringId ConnectionImpl_0_SOCKETTIMEOUTEXCEPTION_DURING_BATCH_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringId(1204, "{0}: SocketTimeoutException during batch operation on region: {1} key: {2} messageId: {3}");
 +  public static final StringId ConnectionImpl_0_SOCKETTIMEOUTEXCEPTION_DURING_CQ_OPERATION = new StringId(1205, "{0}: SocketTimeoutException during cq operation.");
 +  public static final StringId ConnectionImpl_0_SOCKETTIMEOUTEXCEPTION_DURING_DESTROY_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringId(1206, "{0}: SocketTimeoutException during destroy operation on region: {1} key: {2} messageId: {3}");
 +  public static final StringId ConnectionImpl_0_SOCKETTIMEOUTEXCEPTION_DURING_PUT_OPERATION_ON_REGION_1_KEY_2_MESSAGEID_3 = new StringId(1207, "{0}: SocketTimeoutException during put operation on region: {1} key: {2} messageId: {3}");
 +  public static final StringId ConnectionImpl_0_TIMED_OUT_READING_PING_RESPONSE_FOR_TRANSACTION_1 = new StringId(1208, "{0}: Timed out reading ping response for transaction {1}");
 +  public static final StringId ConnectionImpl_AN_INCORRECT_CONNECTION_HAS_BEEN_ASSIGNED_THIS_CONNECTION_IS_ALREADY_PROCESSING_A_REQUEST = new StringId(1209, "An incorrect connection has been assigned. This connection is already processing a request.");
 +  public static final StringId ConnectionImpl_CONNECTION_0_WHILE_PERFORMING_A_REMOTE_CLIENTREADYTORECEIVEUPDATES = new StringId(1210, "Connection ({0}): While performing a remote clientReadyToReceiveUpdates");
 +  public static final StringId ConnectionImpl_THE_CACHESERVER_ON_0_HAS_DIED_UNEXPECTEDLY = new StringId(1211, "The CacheServer on {0} has died unexpectedly");
 +  public static final StringId ConnectionImpl_UNEXPECTED_MESSAGE_TYPE_0 = new StringId(1212, "Unexpected message type {0}");
 +  public static final StringId AttributesFactory_CLONENOTSUPPORTEDEXCEPTION_THROWN_IN_CLASS_THAT_IMPLEMENTS_CLONEABLE = new StringId(1213, "CloneNotSupportedException thrown in class that implements cloneable.");
 +  public static final StringId ConnectionProxyImpl_0_FAILED_TO_SET_PRIMARY_ENDPOINT_TO_1_2 = new StringId(1214, "{0}: Failed to set primary endpoint to <{1}>: {2}");
 +  public static final StringId CqQueryImpl_CQ_IS_CLOSED_CQNAME_0 = new StringId(1215, "CQ is closed, CqName : {0}");
 +  public static final StringId ConnectionProxyImpl_0__ADDING__1 = new StringId(1216, "{0} : Adding  {1}");
 +  public static final StringId ConnectionProxyImpl_0__ATTEMPTING_TO_REMOVE_ENDPOINT__1__2__3 = new StringId(1217, "{0} : Attempting to remove endpoint  {1} -> {2} : {3}");
 +  public static final StringId ConnectionProxyImpl_0__CONFIGURATION__1 = new StringId(1218, "{0} : configuration:  {1}");
 +  public static final StringId ConnectionProxyImpl_0__INITIALIZED_USING__1 = new StringId(1219, "{0} : initialized using:  {1}");
 +  public static final StringId ConnectionProxyImpl_0__LIVE__1__CANNOT_BE_REMOVED_BECAUSE_IT_HAS_CONNECTIONS = new StringId(1220, "{0} : Live  {1}  cannot be removed because it has connections.");
 +  public static final StringId ConnectionProxyImpl_0__LIVE__1__HAS_BEEN_REMOVED = new StringId(1221, "{0} : Live  {1}  has been removed.");
 +  public static final StringId ConnectionProxyImpl_0__NO_EXISTING_LIVE_ENDPOINT_WAS_FOUND_FOR_ENDPOINT__1__2__3__NO_ENDPOINT_WAS_REMOVED = new StringId(1222, "{0} : No existing live endpoint was found for endpoint  {1} -> {2} : {3} . No endpoint was removed.");
 +  public static final StringId ConnectionProxyImpl_0__SUCCESSFULLY_SENT_CLIENT_READY_MESSAGE_AFTER_FAILOVER_TO__1 = new StringId(1223, "{0} : Successfully sent client ready message after failover to  {1}");
 +  public static final StringId ConnectionProxyImpl_0__SUCCESSFULLY_SENT_CLIENT_READY_MESSAGE_TO__1 = new StringId(1224, "{0} : Successfully sent client ready message to  {1}");
 +  public static final StringId ConnectionProxyImpl_0__THE_EXISTING_LIVE_ENDPOINT_NAMED__1__COULD_NOT_BE_REMOVED_BECAUSE_ITS_HOST_AND_PORT__2__3__DOES_NOT_MATCH_THE_REQUESTED_HOST_AND_PORT__4__5_ = new StringId(1225, "{0} : The existing live endpoint named  {1}  could not be removed because its host and port ( {2} : {3} ) does not match the requested host and port ( {4} : {5} ).");
 +  public static final StringId Connection_DISCONNECTED_AS_A_SLOWRECEIVER = new StringId(1226, "Disconnected as a slow-receiver");
 +  public static final StringId ConnectionProxyImpl_CACHE_IS_CLOSED = new StringId(1227, "Cache is closed");
 +  public static final StringId ConnectionProxyImpl_CONDITIONALLYRELEASECONNECTION_FAILED_ON_0_DUE_TO_UNKNOWN_POLICY_TYPE_1 = new StringId(1228, "conditionallyReleaseConnection: failed on <{0}> due to unknown policy type {1}");
 +  public static final StringId ConnectionProxyImpl_CONNECTIONPROXYIMPL_0_FINISHED_INSTANTIATORS_RECOVERY = new StringId(1229, "ConnectionProxyImpl ({0}) finished instantiators recovery.");
 +  public static final StringId ConnectionProxyImpl_CONNECTIONPROXYIMPL_0_STARTED_INSTANTIATORS_RECOVERY = new StringId(1230, "ConnectionProxyImpl ({0}) started instantiators recovery.");
 +  public static final StringId ConnectionProxyImpl_CONNECTIONPROXYIMPL_COULD_NOT_RECOVER_INSTANTIATORS_WITH_0_1 = new StringId(1231, "ConnectionProxyImpl: Could not recover instantiators with {0}. {1}");
 +  public static final StringId ConnectionProxyImpl_COULD_NOT_ACQUIRE_OR_CREATE_A_CONNECTION_NO_ACTIVE_SERVERS_WERE_FOUND_WHILE_REGISTERING_INSTANTIATORS = new StringId(1232, "Could not acquire or create a Connection. No active servers were found while registering instantiators.");
 +  public static final StringId ConnectionProxyImpl_COULD_NOT_ACQUIRE_OR_CREATE_A_CONNECTION_NO_AVAILABLE_CONNECTION_WAS_FOUND_WHILE_REGISTERING_INSTANTIATORS_BUT_THE_FOLLOWING_ACTIVE_SERVERS_EXIST_0 = new StringId(1233, "Could not acquire or create a Connection. No available Connection was found while registering instantiators, but the following active servers exist: {0}");
 +  public static final StringId ConnectionProxyImpl_ERROR_WHILE_REREGISTERING_THE_CQ_TO_THE_REDUNDANT_SERVER_CQNAME_0_ERROR_1 = new StringId(1234, "Error while re-registering the CQ to the redundant server. CqName : {0}, Error : {1}");
 +  public static final StringId ConnectionProxyImpl_EXCEPTION_WHILE_SENDING_AN_ACK_TO_THE_PRIMARY_SERVER = new StringId(1235, "Exception while sending an ack to the primary server");
 +  public static final StringId ConnectionProxyImpl_PROBLEM_REMOVING_ALL_INTEREST_ON_REGION_0_INTERESTTYPE_1 = new StringId(1236, "Problem removing all interest on region={0} interestType={1}");
 +  public static final StringId ConnectionProxyImpl_PROXY_NOT_PROPERLY_INITIALIZED_0 = new StringId(1237, "Proxy not properly initialized: {0}");
 +  public static final StringId ConnectionProxyImpl_THREADIDTOSEQUENCEIDEXPIRYTHREADUNEXPECTED_INTRREUPT_HENCE_CONTINUE_RUNNING = new StringId(1238, "ThreadIdToSequenceIdExpiryThread::Unexpected interrupt, hence continue running");
 +  public static final StringId ConnectionProxyImpl_UNEXPECTED_EXCEPTION_OCCURRED_WHILE_ATTEMPTING_TO_NOTIFY_PRIMARY_SERVER__0__OF_CLIENT_READINESS = new StringId(1239, "Unexpected exception occurred while attempting to notify primary server  {0}  of client readiness");
 +  public static final StringId ConnectionTable_FAILED_TO_ACCEPT_CONNECTION_FROM_0_BECAUSE_1 = new StringId(1240, "Failed to accept connection from {0} because: {1}");
 +  public static final StringId ConnectionTable_KEY_0___VALUE_HASH_1__DESCR_2 = new StringId(1241, "key= {0}   value hash= {1}  descr= {2}");
 +  public static final StringId ConnectionTable_P2P_CONNECTION_TABLE_CONTENTS = new StringId(1242, "P2P connection table contents-----------");
 +  public static final StringId Connection_0_ASYNC_CONFIGURATION_RECEIVED_1 = new StringId(1243, "{0} async configuration received {1}.");
 +  public static final StringId Connection_0_ERROR_READING_MESSAGE = new StringId(1244, "{0} Error reading message");
 +  public static final StringId Connection_0_EXCEPTION_IN_CHANNEL_READ = new StringId(1245, "{0} exception in channel read");
 +  public static final StringId Connection_0_EXCEPTION_RECEIVED = new StringId(1246, "{0} exception received");
 +  public static final StringId Connection_0_STRAY_INTERRUPT_READING_MESSAGE = new StringId(1247, "{0} Stray interrupt reading message");
 +  public static final StringId Connection_0_SUCCESSFULLY_REESTABLISHED_CONNECTION_TO_PEER_1 = new StringId(1248, "{0}: Successfully reestablished connection to peer {1}");
 +  public static final StringId Connection_ACK_READ_EXCEPTION = new StringId(1249, "ack read exception");
 +  public static final StringId Connection_ACK_READ_EXCEPTION_0 = new StringId(1250, "ack read exception: {0}");
 +  public static final StringId Connection_ACK_WAIT_THRESHOLD_EXCEEDED_WAITING_FOR_ACK_FROM_0 = new StringId(1251, "Ack wait threshold exceeded waiting for ack from {0}");
 +  public static final StringId Connection_ACK_WRITE_EXCEPTION = new StringId(1252, "ack write exception");
 +  public static final StringId Connection_ACK_WRITE_EXCEPTION_0 = new StringId(1253, "ack write exception: {0}");
 +  public static final StringId Connection_ALLOCATING_LARGER_NETWORK_READ_BUFFER_NEW_SIZE_IS_0_OLD_SIZE_WAS_1 = new StringId(1254, "Allocating larger network read buffer, new size is {0} old size was {1}.");
 +  public static final StringId Connection_BLOCKED_FOR_0_MS_WHICH_IS_LONGER_THAN_THE_MAX_OF_1_MS_ASKING_SLOW_RECEIVER_2_TO_DISCONNECT = new StringId(1255, "Blocked for {0}ms which is longer than the max of {1

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------



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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/GetMemberInformationFunction.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/GetMemberInformationFunction.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/GetMemberInformationFunction.java
index face825..ff4d945 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/GetMemberInformationFunction.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/GetMemberInformationFunction.java
@@ -25,7 +25,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.internal.InternalEntity;
 import com.gemstone.gemfire.internal.cache.CacheClientStatus;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.management.internal.cli.CliUtil;
 import com.gemstone.gemfire.management.internal.cli.domain.CacheServerInfo;
@@ -114,7 +114,7 @@ public class GetMemberInformationFunction extends FunctionAdapter implements Int
           CacheServerInfo cacheServerInfo = new CacheServerInfo(bindAddress, port, isRunning);
           memberInfo.addCacheServerInfo(cacheServerInfo);
         }
-        Map<ClientProxyMembershipID, CacheClientStatus> allConnectedClients = InternalBridgeMembership
+        Map<ClientProxyMembershipID, CacheClientStatus> allConnectedClients = InternalClientMembership
             .getStatusForAllClientsIgnoreSubscriptionStatus();
         Iterator<ClientProxyMembershipID> it = allConnectedClients.keySet().iterator();
         int numConnections = 0;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/management/membership/ClientMembership.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/membership/ClientMembership.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/membership/ClientMembership.java
index e5369c1..e07f19b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/membership/ClientMembership.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/membership/ClientMembership.java
@@ -7,7 +7,7 @@
  */
 package com.gemstone.gemfire.management.membership;
 
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
+import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 
 /**
  * Provides utility methods for registering and unregistering
@@ -29,7 +29,7 @@ public final class ClientMembership {
    *          a ClientMembershipListener to be registered
    */
   public static void registerClientMembershipListener(ClientMembershipListener listener) {
-    InternalBridgeMembership.registerClientMembershipListener(listener);
+    InternalClientMembership.registerClientMembershipListener(listener);
   }
 
   /**
@@ -40,7 +40,7 @@ public final class ClientMembership {
    *          a ClientMembershipListener to be unregistered
    */
   public static void unregisterClientMembershipListener(ClientMembershipListener listener) {
-    InternalBridgeMembership.unregisterClientMembershipListener(listener);
+    InternalClientMembership.unregisterClientMembershipListener(listener);
   }
 
   /**
@@ -52,7 +52,7 @@ public final class ClientMembership {
    *         array if no listeners
    */
   public static ClientMembershipListener[] getClientMembershipListeners() {
-    return InternalBridgeMembership.getClientMembershipListeners();
+    return InternalClientMembership.getClientMembershipListeners();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/resources/META-INF/schemas/schema.pivotal.io/gemfire/cache/cache-9.0.xsd
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/resources/META-INF/schemas/schema.pivotal.io/gemfire/cache/cache-9.0.xsd b/gemfire-core/src/main/resources/META-INF/schemas/schema.pivotal.io/gemfire/cache/cache-9.0.xsd
index 35af50c..989ca37 100644
--- a/gemfire-core/src/main/resources/META-INF/schemas/schema.pivotal.io/gemfire/cache/cache-9.0.xsd
+++ b/gemfire-core/src/main/resources/META-INF/schemas/schema.pivotal.io/gemfire/cache/cache-9.0.xsd
@@ -257,15 +257,6 @@ declarative caching XML file elements unless indicated otherwise.
             </xsd:complexContent>
           </xsd:complexType>
         </xsd:element>
-        <xsd:element maxOccurs="unbounded" minOccurs="0" name="bridge-server" type="gf:server-type">
-          <xsd:annotation>
-            <xsd:documentation>
-              The "bridge-server" element specifies a Cache Bridge Server that is started when the Cache is declaratively initialized.
-              NOTE: as
-              of 5.7 this element is deprecated; use the "cache-server" element instead.
-            </xsd:documentation>
-          </xsd:annotation>
-        </xsd:element>
         <xsd:element maxOccurs="unbounded" minOccurs="0" name="pool" type="gf:pool-type" />
         <xsd:element maxOccurs="unbounded" minOccurs="0" name="disk-store" type="gf:disk-store-type" />
         <xsd:element maxOccurs="unbounded" minOccurs="0" name="hdfs-store" type="gf:hdfs-store-type" />
@@ -917,14 +908,6 @@ As of 6.5 disk-dirs is deprecated on region-attributes. Use disk-store-name inst
     <!-- ======================================================= -->
 
   <xsd:complexType name="server-type">
-    <xsd:annotation>
-      <xsd:documentation>
-        The "bridge-server" element specifies a Cache Bridge Server that is started when the Cache is declaratively initialized.
-        NOTE: as
-        of 5.7
-        this element is deprecated; use the "cache-server" element instead.
-      </xsd:documentation>
-    </xsd:annotation>
     <xsd:sequence>
       <xsd:element maxOccurs="unbounded" minOccurs="0" name="group" type="xsd:string">
         <xsd:annotation>
@@ -1327,19 +1310,10 @@ As of 6.5 disk-dirs is deprecated on region-attributes. Use disk-store-name inst
         pool used
         by client applications in a client/server cache configuration.
         It should not be specified in servers or peers.
-
-        The optional "cache-loader" and
-        "cache-writer" are for legacy client client
-        applications that doen't use a connection pool. They must be a
-        bridge-loader and a bridge-writer. They should
-        not be specified in
-        servers or peers.
       </xsd:documentation>
     </xsd:annotation>
     <xsd:sequence>
       <xsd:element maxOccurs="1" minOccurs="0" name="disk-dir" type="gf:disk-dir-type" />
-      <xsd:element maxOccurs="1" minOccurs="0" name="cache-loader" type="gf:cache-loader-type" />
-      <xsd:element maxOccurs="1" minOccurs="0" name="cache-writer" type="gf:cache-writer-type" />
     </xsd:sequence>
     <xsd:attribute name="disable-persist-backup" type="xsd:boolean" use="optional" />
     <xsd:attribute name="disable-register-interest" type="xsd:boolean" use="optional" />

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/batterytest/greplogs/ExpectedStrings.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/batterytest/greplogs/ExpectedStrings.java b/gemfire-core/src/test/java/batterytest/greplogs/ExpectedStrings.java
index 1888fbe..ec4d4a4 100644
--- a/gemfire-core/src/test/java/batterytest/greplogs/ExpectedStrings.java
+++ b/gemfire-core/src/test/java/batterytest/greplogs/ExpectedStrings.java
@@ -120,8 +120,7 @@ public class ExpectedStrings {
       expected.add(Pattern.compile("No admin on"));
       expected.add(Pattern.compile("nonExistentMethod"));
       expected.add(Pattern.compile("Expected exception"));
-      expected.add(Pattern.compile("BridgeLoaderTestNonSerializable"));
-      expected.add(Pattern.compile("BridgeLoaderTestSerializableImpl"));
+      expected.add(Pattern.compile("ConnectionPoolTestNonSerializable"));
       expected.add(Pattern.compile("One or more DUnit tests failed"));
       expected.add(Pattern.compile("ReplyException"));
       expected.add(Pattern.compile("fine 2"));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java
index e2e5024..54482d0 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/AttributesFactoryJUnitTest.java
@@ -14,7 +14,6 @@ import org.junit.experimental.categories.Category;
 
 import junit.framework.TestCase;
 
-import com.gemstone.gemfire.cache.util.BridgeLoader;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.cache.util.CacheWriterAdapter;
 import com.gemstone.gemfire.compression.SnappyCompressor;
@@ -400,23 +399,6 @@ public class AttributesFactoryJUnitTest extends TestCase {
     factory = new AttributesFactory();
     factory.setCacheLoader(cl);
     factory.setPoolName("mypool");
-    
-    factory = new AttributesFactory();
-    factory.setCacheLoader(new BridgeLoader());
-    
-    try {
-      factory.setPoolName("mypool");
-      fail("expected IllegalStateException");
-    } catch (IllegalStateException expected) {
-    }
-    
-    factory = new AttributesFactory();
-    factory.setCacheLoader(new BridgeLoader());
-    try {
-      factory.setPoolName("mypool");
-      fail("expected IllegalStateException");
-    } catch (IllegalStateException expected) {
-    }
   }
   
  /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientHelper.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientHelper.java
index 29c3d41..c961159 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientHelper.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ClientHelper.java
@@ -14,7 +14,6 @@ import java.util.Set;
 import com.gemstone.gemfire.cache.DynamicRegionFactory.Config;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.ServerProxy;
-import com.gemstone.gemfire.cache.util.BridgeWriter;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolAndLoaderDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolAndLoaderDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolAndLoaderDUnitTest.java
index ec8fb09..336dfd3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolAndLoaderDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/ConnectionPoolAndLoaderDUnitTest.java
@@ -14,7 +14,7 @@ import junit.framework.Assert;
 
 import com.gemstone.gemfire.cache.client.PoolFactory;
 import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache.util.CacheWriterAdapter;
 import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -394,7 +394,7 @@ public class ConnectionPoolAndLoaderDUnitTest  extends CacheTestCase {
   throws IOException {
 
   Cache cache = getCache();
-  BridgeServer bridge = cache.addBridgeServer();
+  CacheServer bridge = cache.addCacheServer();
   bridge.setPort(port);
   bridge.setNotifyBySubscription(notifyBySubscription);
   bridge.start();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java
index 1fdb67c..0e63bff 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceDUnitTest.java
@@ -20,12 +20,12 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.NoAvailableServersException;
 import com.gemstone.gemfire.cache.client.PoolManager;
 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.BridgeMembershipListenerAdapter;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.management.membership.ClientMembership;
+import com.gemstone.gemfire.management.membership.ClientMembershipEvent;
+import com.gemstone.gemfire.management.membership.ClientMembershipListenerAdapter;
 
 import dunit.Host;
 import dunit.SerializableCallable;
@@ -269,7 +269,7 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
     final String locators = getServerHostName(vm0.getHost()) + "[" + locatorPort + "]";
     
     final int serverPort1 =startBridgeServerInVM(vm1, new String[] {"group1"}, locators);
-    final int serverPort2 =addBridgeServerInVM(vm1, new String[] {"group2"});
+    final int serverPort2 =addCacheServerInVM(vm1, new String[] {"group2"});
     
     startBridgeClientInVM(vm2, "group2", getServerHostName(vm0.getHost()), locatorPort);
     
@@ -282,7 +282,7 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
     checkEndpoints(vm2, new int[] {serverPort1});
   }
   
-  public void testBridgeMembershipListener() throws Exception {
+  public void testClientMembershipListener() throws Exception {
     final Host host = Host.getHost(0);
     VM locatorVM = host.getVM(0);
     VM bridge1VM = host.getVM(1);
@@ -452,7 +452,7 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
     vm.invoke(new SerializableRunnable("Add membership listener") {
       public void run() {
         MyListener listener = new MyListener();
-        BridgeMembership.registerBridgeMembershipListener(listener);
+        ClientMembership.registerClientMembershipListener(listener);
         remoteObjects.put(BRIDGE_LISTENER, listener);
       }
     });
@@ -541,12 +541,13 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
     });
   }
   
-  public static class MyListener extends BridgeMembershipListenerAdapter implements Serializable {
+  public static class MyListener extends ClientMembershipListenerAdapter implements Serializable {
     protected int crashes = 0;
     protected int joins = 0;
     protected int departures= 0;
 
-    public synchronized void memberCrashed(BridgeMembershipEvent event) {
+    @Override
+    public synchronized void memberCrashed(ClientMembershipEvent event) {
       crashes++;
       notifyAll();
     }
@@ -557,12 +558,14 @@ public class AutoConnectionSourceDUnitTest extends LocatorTestBase {
       departures = 0;
     }
 
-    public synchronized void memberJoined(BridgeMembershipEvent event) {
+    @Override
+    public synchronized void memberJoined(ClientMembershipEvent event) {
       joins++;
       notifyAll();
     }
 
-    public synchronized void memberLeft(BridgeMembershipEvent event) {
+    @Override
+    public synchronized void memberLeft(ClientMembershipEvent event) {
       departures++;
       notifyAll();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceWithUDPDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceWithUDPDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceWithUDPDUnitTest.java
index 8509c54..9f1d67a 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceWithUDPDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/AutoConnectionSourceWithUDPDUnitTest.java
@@ -21,7 +21,7 @@ import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -90,7 +90,7 @@ public class AutoConnectionSourceWithUDPDUnitTest extends
             for(int i = 0; i < regions.length; i++) {
               cache.createRegion(regions[i], attrs);
             }
-            BridgeServer server = cache.addBridgeServer();
+            CacheServer server = cache.addCacheServer();
             final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
             server.setPort(serverPort);
             server.setGroups(groups);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ConnectionPoolImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ConnectionPoolImplJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ConnectionPoolImplJUnitTest.java
index ab67729..dcede9b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ConnectionPoolImplJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/ConnectionPoolImplJUnitTest.java
@@ -24,10 +24,7 @@ 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.ServerConnectivityException;
-import com.gemstone.gemfire.cache.util.BridgeServer;
-import com.gemstone.gemfire.cache.util.EndpointDoesNotExistException;
-import com.gemstone.gemfire.cache.util.EndpointExistsException;
-import com.gemstone.gemfire.cache.util.EndpointInUseException;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -158,8 +155,8 @@ public class ConnectionPoolImplJUnitTest {
   
   @Test
   public void testExecuteOp() throws Exception {
-    BridgeServer server1 = cache.addBridgeServer();
-    BridgeServer server2 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
+    CacheServer server2 = cache.addCacheServer();
     int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(2);
     int port1 = ports[0];
     int port2 = ports[1];
@@ -223,7 +220,7 @@ public class ConnectionPoolImplJUnitTest {
   
   @Test
   public void testCreatePool() throws Exception {
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     int port1 = port;
     server1.setPort(port1);
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java
index dffc216..ac6f695 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorLoadBalancingDUnitTest.java
@@ -27,14 +27,14 @@ import com.gemstone.gemfire.cache.client.internal.locator.QueueConnectionRespons
 import com.gemstone.gemfire.cache.server.ServerLoad;
 import com.gemstone.gemfire.cache.server.ServerLoadProbeAdapter;
 import com.gemstone.gemfire.cache.server.ServerMetrics;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.distributed.internal.ServerLocator;
 import com.gemstone.gemfire.distributed.internal.tcpserver.TcpClient;
 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.PoolFactoryImpl;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
@@ -241,8 +241,8 @@ public class LocatorLoadBalancingDUnitTest extends LocatorTestBase {
     Runnable checkConnectionCount = new SerializableRunnable("checkConnectionCount") {
       public void run() {
         Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
-        final BridgeServerImpl server = (BridgeServerImpl)
-            cache.getBridgeServers().get(0);
+        final CacheServerImpl server = (CacheServerImpl)
+            cache.getCacheServers().get(0);
         WaitCriterion wc = new WaitCriterion() {
           String excuse;
           public boolean done() {
@@ -456,7 +456,7 @@ public class LocatorLoadBalancingDUnitTest extends LocatorTestBase {
 
       public void run() {
         Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
-        BridgeServer server = (BridgeServer) cache.getBridgeServers().get(0);
+        CacheServer server = (CacheServer) cache.getCacheServers().get(0);
         MyLoadProbe probe = (MyLoadProbe) server.getLoadProbe();
         probe.setLoad(newLoad);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java
index 0776112..01a9c19 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/internal/LocatorTestBase.java
@@ -27,7 +27,7 @@ import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.Pool;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.server.ServerLoadProbe;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.Locator;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -135,13 +135,13 @@ public abstract class LocatorTestBase  extends DistributedTestCase {
     return startBridgeServerInVM(vm, groups, locators, new String[] {REGION_NAME});
   }
   
-  protected int addBridgeServerInVM(VM vm, final String[] groups) {
+  protected int addCacheServerInVM(VM vm, final String[] groups) {
     SerializableCallable connect =
       new SerializableCallable("Add Bridge server") {
 
       public Object call() throws Exception {
         Cache cache = (Cache) remoteObjects.get(CACHE_KEY);
-        BridgeServer server = cache.addBridgeServer();
+        CacheServer server = cache.addCacheServer();
         final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
         server.setPort(serverPort);
         server.setGroups(groups);
@@ -154,7 +154,7 @@ public abstract class LocatorTestBase  extends DistributedTestCase {
   }
   
   protected int startBridgeServerInVM(VM vm, final String[] groups, final String locators, final String[] regions) {
-    return startBridgeServerInVM(vm, groups, locators, regions, BridgeServer.DEFAULT_LOAD_PROBE);
+    return startBridgeServerInVM(vm, groups, locators, regions, CacheServer.DEFAULT_LOAD_PROBE);
   }
   
   protected int startBridgeServerInVM(VM vm, final String[] groups, final String locators, final String[] regions, final ServerLoadProbe probe) {
@@ -174,7 +174,7 @@ public abstract class LocatorTestBase  extends DistributedTestCase {
             for(int i = 0; i < regions.length; i++) {
               cache.createRegion(regions[i], attrs);
             }
-            BridgeServer server = cache.addBridgeServer();
+            CacheServer server = cache.addCacheServer();
             final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();
             server.setPort(serverPort);
             server.setGroups(groups);
@@ -208,7 +208,7 @@ public abstract class LocatorTestBase  extends DistributedTestCase {
             for(int i = 0; i < regions.length; i++) {
               cache.createRegion(regions[i], attrs);
             }
-            BridgeServer server = cache.addBridgeServer();
+            CacheServer server = cache.addCacheServer();
             server.setGroups(groups);
             server.setLoadProbe(probe);
             final int serverPort = AvailablePortHelper.getRandomAvailableTCPPort();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java
index 37f9744..8c4f893 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsDUnitTest.java
@@ -47,7 +47,7 @@ import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
 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.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -84,7 +84,7 @@ import dunit.VM;
  * @author Mitch Thomas
  * @since 6.0
  */
-public class MemoryThresholdsDUnitTest extends BridgeTestCase {
+public class MemoryThresholdsDUnitTest extends ClientServerTestCase {
   
   public static class Range implements Serializable {
     public final static Range DEFAULT = new Range(0, 20);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
index 83b8608..d65dcc7 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
@@ -35,7 +35,7 @@ import com.gemstone.gemfire.cache.client.ServerOperationException;
 import com.gemstone.gemfire.cache.control.ResourceManager;
 import com.gemstone.gemfire.cache.management.MemoryThresholdsDUnitTest.Range;
 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.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -72,7 +72,7 @@ import dunit.VM;
  * @author David Hoots
  * @since 9.0
  */
-public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
+public class MemoryThresholdsOffHeapDUnitTest extends ClientServerTestCase {
   private static final long serialVersionUID = -684231183212051910L;
 
   final String expectedEx = LocalizedStrings.MemoryMonitor_MEMBER_ABOVE_CRITICAL_THRESHOLD.getRawText().replaceAll("\\{[0-9]+\\}",

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java
index 15ab005..bf5a150 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/PdxStringQueryDUnitTest.java
@@ -45,7 +45,7 @@ import com.gemstone.gemfire.cache.query.internal.index.RangeIndex;
 import com.gemstone.gemfire.cache.query.types.CollectionType;
 import com.gemstone.gemfire.cache.query.types.ObjectType;
 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.internal.AvailablePortHelper;
@@ -168,7 +168,7 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
       public void run2() throws CacheException {
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
         Region region = createRegion(regionName, rootRegionName,  factory.create());
         getLogWriter().info("Put PortfolioPdx");
         for (int i=0; i<numberOfEntries; i++) {
@@ -426,7 +426,7 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
       public void run2() throws CacheException {
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
         Region region = createRegion(regionName, rootRegionName,  factory.create());
         getLogWriter().info("Put PortfolioPdx");
         for (int i=0; i<numberOfEntries; i++) {
@@ -638,7 +638,7 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
       public void run2() throws CacheException {
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
         Region region = createRegion(regionName, rootRegionName,  factory.create());
         getLogWriter().info("Put PortfolioPdx");
         for (int i=0; i<numberOfEntries; i++) {
@@ -852,7 +852,7 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
       public void run2() throws CacheException {
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
         Region region = createRegion(regionName, rootRegionName,  factory.create());
         getLogWriter().info("Put PortfolioPdx");
         for (int i=0; i<numberOfEntries; i++) {
@@ -1116,7 +1116,7 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
       public void run2() throws CacheException {
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
         Region region = createRegion(regionName, rootRegionName,  factory.create());
         getLogWriter().info("Put PortfolioPdx");
         for (int i=0; i<numberOfEntries; i++) {
@@ -1343,7 +1343,7 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
       public void run2() throws CacheException {
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
         Region region = createRegion(regionName, rootRegionName,  factory.create());
         getLogWriter().info("Put PortfolioPdx");
         for (int i=0; i<numberOfEntries; i++) {
@@ -1570,7 +1570,7 @@ public class PdxStringQueryDUnitTest extends CacheTestCase{
       public void run2() throws CacheException {
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, host0, port1,-1, true, -1, -1, null);
         Region region = createRegion(regionName, rootRegionName,  factory.create());
       
         getLogWriter().info("Put PortfolioPdx");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
index 0e982bf..68ec0c5 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryUsingPoolDUnitTest.java
@@ -32,8 +32,8 @@ import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.cache.query.internal.QueryObserverAdapter;
 import com.gemstone.gemfire.cache.query.internal.QueryObserverHolder;
 import com.gemstone.gemfire.cache.query.types.ObjectType;
-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.cache30.CacheTestCase;
 import com.gemstone.gemfire.distributed.DistributedSystem;
@@ -575,7 +575,7 @@ public class QueryUsingPoolDUnitTest extends CacheTestCase {
         getCache();
         AttributesFactory factory = new AttributesFactory();
         factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
+        ClientServerTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
         createRegion(name, factory.create());
       }
     });
@@ -2421,7 +2421,7 @@ public class QueryUsingPoolDUnitTest extends CacheTestCase {
   throws IOException {
 
     Cache cache = getCache();
-    BridgeServer bridge = cache.addBridgeServer();
+    CacheServer bridge = cache.addCacheServer();
     bridge.setPort(port);
     bridge.setNotifyBySubscription(notifyBySubscription);
     bridge.start();
@@ -2432,8 +2432,8 @@ public class QueryUsingPoolDUnitTest extends CacheTestCase {
    * Stops the bridge server that serves up the given cache.
    */
   protected void stopBridgeServer(Cache cache) {
-    BridgeServer bridge =
-      (BridgeServer) cache.getBridgeServers().iterator().next();
+    CacheServer bridge =
+      (CacheServer) cache.getCacheServers().iterator().next();
     bridge.stop();
     assertFalse(bridge.isRunning());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
index 0028540..ef24789 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/RemoteQueryDUnitTest.java
@@ -10,10 +10,11 @@ package com.gemstone.gemfire.cache.query.dunit;
 import com.gemstone.gemfire.DataSerializable;
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.*;
+import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.query.*;
 import com.gemstone.gemfire.cache.query.internal.*;
-import com.gemstone.gemfire.cache.util.*;
-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.cache30.CacheTestCase;
 import com.gemstone.gemfire.distributed.DistributedSystem;
@@ -43,9 +44,6 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
   /** The port on which the bridge server was started in this VM */
   private static int bridgeServerPort;
 
-  /**
-   * Creates a new <code>GemFireMemberStatusDUnitTest</code>
-   */
   public RemoteQueryDUnitTest(String name) {
     super(name);
   }
@@ -119,7 +117,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           getCache();
           AttributesFactory factory = new AttributesFactory();
           factory.setScope(Scope.LOCAL);
-          BridgeTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
+          ClientServerTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
           createRegion(name, factory.create());
         }
       });
@@ -264,7 +262,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           AttributesFactory factory = new AttributesFactory();
           factory.setScope(Scope.LOCAL);
           
-          BridgeTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
+          ClientServerTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
           createRegion(name, factory.create());
         }
       });
@@ -392,7 +390,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           getCache();
           AttributesFactory factory = new AttributesFactory();
           factory.setScope(Scope.LOCAL);
-          BridgeTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
+          ClientServerTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
           createRegion(name, factory.create());
         }
       });
@@ -626,7 +624,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           getCache();
           AttributesFactory factory = new AttributesFactory();
           factory.setScope(Scope.LOCAL);
-          BridgeTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
+          ClientServerTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
           createRegion(name, factory.create());
         }
       });
@@ -714,7 +712,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           getCache();
           AttributesFactory factory = new AttributesFactory();
           factory.setScope(Scope.LOCAL);
-          BridgeTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
+          ClientServerTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
           createRegion(name, factory.create());
         }
       });
@@ -885,7 +883,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           getCache();
           AttributesFactory factory = new AttributesFactory();
           factory.setScope(Scope.LOCAL);
-          BridgeTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
+          ClientServerTestCase.configureConnectionPool(factory, host0, port,-1, true, -1, -1, null);
           createRegion(name+"1", factory.create());
           createRegion(name+"2", factory.create());
         }
@@ -952,7 +950,6 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           AttributesFactory factory = new AttributesFactory();
           factory.setScope(Scope.LOCAL);
           createRegion(name, factory.create());
-          pause(1000);
           try {
             startBridgeServer(0, false);
           } catch (Exception ex) {
@@ -980,16 +977,11 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           Properties config = new Properties();
           config.setProperty("mcast-port", "0");
           system = (InternalDistributedSystem) DistributedSystem.connect(config);
+          PoolManager.createFactory().addServer(host0, port).setSubscriptionEnabled(true).create("clientPool");
           getCache();
           AttributesFactory factory = new AttributesFactory();
           factory.setScope(Scope.LOCAL);
-          BridgeClient writer = new BridgeClient();
-          Properties props = new Properties();
-          props.setProperty("endpoints", "server=" + host0 + ":" +
-                            port);
-          props.setProperty("establishCallbackConnection", "true");
-          writer.init(props);
-          factory.setCacheWriter(writer);
+          factory.setPoolName("clientPool");
           createRegion(name, factory.create());
         }
       });
@@ -1000,16 +992,11 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
           Properties config = new Properties();
           config.setProperty("mcast-port", "0");
           system = (InternalDistributedSystem) DistributedSystem.connect(config);
+          PoolManager.createFactory().addServer(host0, port).setSubscriptionEnabled(true).create("clientPool");
           getCache();
           AttributesFactory factory = new AttributesFactory();
           factory.setScope(Scope.LOCAL);
-          BridgeClient loader = new BridgeClient();
-          Properties props = new Properties();
-          props.setProperty("endpoints", "server=" + host0 + ":" +
-                            port);
-          props.setProperty("establishCallbackConnection", "true");
-          loader.init(props);
-          factory.setCacheLoader(loader);
+          factory.setPoolName("clientPool");
           createRegion(name, factory.create());
         }
       });
@@ -1076,10 +1063,8 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
       vm1.invoke(new CacheSerializableRunnable("Close client") {
         public void run2() throws CacheException {
           Region region = getRootRegion().getSubregion(name);
-          BridgeClient writer = (BridgeClient)
-            region.getAttributes().getCacheWriter();
-          writer.close();
-          region.getAttributesMutator().setCacheWriter(null);
+          region.close();
+          PoolManager.find("clientPool").destroy();
         }
       });
 
@@ -1087,10 +1072,8 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
       vm2.invoke(new CacheSerializableRunnable("Close client") {
         public void run2() throws CacheException {
           Region region = getRootRegion().getSubregion(name);
-          BridgeClient loader = (BridgeClient)
-            region.getAttributes().getCacheLoader();
-          loader.close();
-          region.getAttributesMutator().setCacheLoader(null);
+          region.close();
+          PoolManager.find("clientPool").destroy();
         }
       });
 
@@ -1127,7 +1110,6 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
            AttributesFactory factory = new AttributesFactory();
            factory.setScope(Scope.LOCAL);
            createRegion(name, factory.createRegionAttributes());
-           pause(1000);
            try {
              startBridgeServer(0, false);
            } catch (Exception ex) {
@@ -1155,16 +1137,11 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
            Properties config = new Properties();
            config.setProperty("mcast-port", "0");
            system = (InternalDistributedSystem) DistributedSystem.connect(config);
+           PoolManager.createFactory().addServer(host0, port).setSubscriptionEnabled(true).create("clientPool");
            getCache();
            AttributesFactory factory = new AttributesFactory();
            factory.setScope(Scope.LOCAL);
-           BridgeClient writer = new BridgeClient();
-           Properties props = new Properties();
-           props.setProperty("endpoints", "server=" + host0 + ":" +
-                             port);
-           props.setProperty("establishCallbackConnection", "true");
-           writer.init(props);
-           factory.setCacheWriter(writer);
+           factory.setPoolName("clientPool");
            createRegion(name, factory.createRegionAttributes());
          }
        });
@@ -1197,10 +1174,8 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
        vm1.invoke(new CacheSerializableRunnable("Close client") {
          public void run2() throws CacheException {
            Region region = getRootRegion().getSubregion(name);
-           BridgeClient writer = (BridgeClient)
-             region.getAttributes().getCacheWriter();
-           writer.close();
-           region.getAttributesMutator().setCacheWriter(null);
+           region.close();
+           PoolManager.find("clientPool").destroy();
          }
        });
 
@@ -1247,7 +1222,6 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
               }
 
             });
-            pause(1000);
             try {
               startBridgeServer(0, false);
             } catch (Exception ex) {
@@ -1275,16 +1249,11 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
             Properties config = new Properties();
             config.setProperty("mcast-port", "0");
             system = (InternalDistributedSystem) DistributedSystem.connect(config);
+            PoolManager.createFactory().addServer(host0, port).setSubscriptionEnabled(true).create("clientPool");
             getCache();
             AttributesFactory factory = new AttributesFactory();
             factory.setScope(Scope.LOCAL);
-            BridgeClient writer = new BridgeClient();
-            Properties props = new Properties();
-            props.setProperty("endpoints", "server=" + host0 + ":" +
-                              port);
-            props.setProperty("establishCallbackConnection", "true");
-            writer.init(props);
-            factory.setCacheWriter(writer);
+            factory.setPoolName("clientPool");
             createRegion(name, factory.createRegionAttributes());
           }
         });
@@ -1321,7 +1290,6 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
               }
 
             });
-            pause(1000);
             for (int i=0; i<numberOfEntries; i++) {
               region1.put("key-"+i, new TestObject(i, "ibm"));
               region2.put("key-"+i, new TestObject(i, "ibm"));
@@ -1351,10 +1319,8 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
         vm1.invoke(new CacheSerializableRunnable("Close client") {
           public void run2() throws CacheException {
             Region region = getRootRegion().getSubregion(name);
-            BridgeClient writer = (BridgeClient)
-              region.getAttributes().getCacheWriter();
-            writer.close();
-            region.getAttributesMutator().setCacheWriter(null);
+            region.close();
+            PoolManager.find("clientPool").destroy();
           }
         });
 
@@ -1382,7 +1348,7 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
     throws IOException {
 
     Cache cache = getCache();
-    BridgeServer bridge = cache.addBridgeServer();
+    CacheServer bridge = cache.addCacheServer();
     bridge.setPort(port);
     bridge.setNotifyBySubscription(notifyBySubscription);
     bridge.start();
@@ -1393,8 +1359,8 @@ public class RemoteQueryDUnitTest extends CacheTestCase {
    * Stops the bridge server that serves up the given cache.
    */
   protected void stopBridgeServer(Cache cache) {
-    BridgeServer bridge =
-      (BridgeServer) cache.getBridgeServers().iterator().next();
+    CacheServer bridge =
+      (CacheServer) cache.getCacheServers().iterator().next();
     bridge.stop();
     assertFalse(bridge.isRunning());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
index cdd03b3..4cd2439 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/ResourceManagerWithQueryMonitorDUnitTest.java
@@ -42,7 +42,7 @@ import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.cache.query.internal.QueryMonitor;
 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.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
@@ -63,7 +63,7 @@ import dunit.Host;
 import dunit.SerializableCallable;
 import dunit.VM;
 
-public class ResourceManagerWithQueryMonitorDUnitTest extends BridgeTestCase {
+public class ResourceManagerWithQueryMonitorDUnitTest extends ClientServerTestCase {
   
   private static int MAX_TEST_QUERY_TIMEOUT = 4000;
   private static int TEST_QUERY_TIMEOUT = 1000;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexInitOnOverflowRegionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexInitOnOverflowRegionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexInitOnOverflowRegionDUnitTest.java
index 24f6d64..0ff36c1 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexInitOnOverflowRegionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/ConcurrentIndexInitOnOverflowRegionDUnitTest.java
@@ -31,7 +31,7 @@ import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.cache.query.data.PortfolioData;
 import com.gemstone.gemfire.cache.query.internal.index.IndexManager.TestHook;
 import com.gemstone.gemfire.cache.query.partitioned.PRQueryDUnitHelper;
-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.internal.cache.EvictionAttributesImpl;
@@ -200,7 +200,7 @@ public class ConcurrentIndexInitOnOverflowRegionDUnitTest extends CacheTestCase
         Region partitionRegion = null;
         IndexManager.testHook = null;
         try {
-          BridgeServer bridge = cache.addBridgeServer();
+          CacheServer bridge = cache.addCacheServer();
           bridge.setPort(0);
           bridge.start();
           bridgeServerPort = bridge.getPort();


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeMembershipDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeMembershipDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeMembershipDUnitTest.java
deleted file mode 100644
index ca7a439..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeMembershipDUnitTest.java
+++ /dev/null
@@ -1,1642 +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.cache30;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-import com.gemstone.gemfire.InternalGemFireException;
-import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.Statistics;
-import com.gemstone.gemfire.StatisticsType;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.client.ClientCache;
-import com.gemstone.gemfire.cache.client.Pool;
-import com.gemstone.gemfire.cache.client.PoolManager;
-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.distributed.DistributedMember;
-import com.gemstone.gemfire.distributed.DurableClientAttributes;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.cache.tier.InternalBridgeMembership;
-import com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl;
-import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
-import com.gemstone.gemfire.internal.logging.LocalLogWriter;
-import com.gemstone.gemfire.internal.logging.InternalLogWriter;
-
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.SerializableRunnable;
-import dunit.VM;
-import dunit.DistributedTestCase.WaitCriterion;
-
-/**
- * Tests the BridgeMembership API including BridgeMembershipListener.
- *
- * @author Kirk Lund
- * @since 4.2.1
- */
-public class BridgeMembershipDUnitTest extends BridgeTestCase {
-
-  protected static final boolean CLIENT = true;
-  protected static final boolean SERVER = false;
-  
-  protected static final int JOINED = 0;
-  protected static final int LEFT = 1;
-  protected static final int CRASHED = 2;
-    
-  public BridgeMembershipDUnitTest(String name) {
-    super(name);
-  }
-
-  public void setUp() throws Exception {
-    super.setUp();
-    getSystem();
-  }
-  
-  public void tearDown2() throws Exception {
-    super.tearDown2();
-    InternalBridgeMembership.unregisterAllListeners();
-  }
-
-  private void waitForAcceptsInProgressToBe(final int target)
-    throws Exception {
-    WaitCriterion ev = new WaitCriterion() {
-      String excuse;
-      public boolean done() {
-        int actual = getAcceptsInProgress();
-        if (actual == getAcceptsInProgress()) {
-          return true;
-        }
-        excuse = "accepts in progress (" + actual + ") never became " + target;
-        return false;
-      }
-      public String description() {
-        return excuse;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
-  }
-  
-  protected int getAcceptsInProgress() {
-    StatisticsType st = InternalDistributedSystem.getAnyInstance().findType("CacheServerStats");
-    Statistics[] s = InternalDistributedSystem.getAnyInstance().findStatisticsByType(st);
-    return s[0].getInt("acceptsInProgress");
-  }
-
-  protected static Socket meanSocket;
-
-  /** test that a server times out waiting for a handshake that
-      never arrives. 
-   */
-  public void testConnectionTimeout() throws Exception {
-    addExpectedException("failed accepting client connection");
-    final Host host = Host.getHost(0);
-    final String hostName = getServerHostName(host);
-    final VM vm0 = host.getVM(0);
-    System.setProperty(AcceptorImpl.ACCEPT_TIMEOUT_PROPERTY_NAME, "1000");
-    try {
-    final int port = startBridgeServer(0);
-//    AsyncInvocation ai = null;
-    try {
-      assertTrue(port != 0);
-      SerializableRunnable createMeanSocket = new CacheSerializableRunnable("Connect to server with socket") {
-        public void run2() throws CacheException {
-          getCache(); // create a cache so we have stats
-          getLogWriter().info("connecting to cache server with socket");
-          try {
-            InetAddress addr = InetAddress.getByName(hostName);
-            meanSocket = new Socket(addr, port);
-          }
-          catch (Exception e) {
-            throw new RuntimeException("Test failed to connect or was interrupted", e);
-          }
-        }
-      };
-      SerializableRunnable closeMeanSocket = new CacheSerializableRunnable("close mean socket") {
-        public void run2() throws CacheException {
-          getLogWriter().info("closing mean socket");
-          try {
-            meanSocket.close();
-          }
-          catch (IOException ignore) {
-          }
-        }
-      };
-
-      assertEquals(0, getAcceptsInProgress());
-      
-      getLogWriter().info("creating mean socket");
-      vm0.invoke(createMeanSocket);
-      try {
-        getLogWriter().info("waiting to see it connect on server");
-        waitForAcceptsInProgressToBe(1);
-      } finally {
-        getLogWriter().info("closing mean socket");
-        vm0.invoke(closeMeanSocket);
-      }
-      getLogWriter().info("waiting to see accept to go away on server");
-      waitForAcceptsInProgressToBe(0);
-
-      // now try it without a close. Server should timeout the mean connect
-      getLogWriter().info("creating mean socket 2");
-      vm0.invoke(createMeanSocket);
-      try {
-        getLogWriter().info("waiting to see it connect on server 2");
-        waitForAcceptsInProgressToBe(1);
-        getLogWriter().info("waiting to see accept to go away on server without us closing");
-        waitForAcceptsInProgressToBe(0);
-      } finally {
-        getLogWriter().info("closing mean socket 2");
-        vm0.invoke(closeMeanSocket);
-      }
-
-//       SerializableRunnable denialOfService = new CacheSerializableRunnable("Do lots of connects") {
-//         public void run2() throws CacheException {
-//           int connectionCount = 0;
-//           ArrayList al = new ArrayList(60000);
-//           try {
-//             InetAddress addr = InetAddress.getLocalHost();
-//             for (;;) {
-//               Socket s = new Socket(addr, port);
-//               al.add(s);
-//               connectionCount++;
-//               getLogWriter().info("connected # " + connectionCount + " s=" + s);
-// //               try {
-// //                 s.close();
-// //               } catch (IOException ignore) {}
-//             }
-//           }
-//           catch (Exception e) {
-//             getLogWriter().info("connected # " + connectionCount
-//                                 + " stopped because of exception " + e);
-//             Iterator it = al.iterator();
-//             while (it.hasNext()) {
-//               Socket s = (Socket)it.next();
-//               try {
-//                 s.close();
-//               } catch (IOException ignore) {}
-//             }
-//           }
-//         }
-//       };
-//       // now pretend to do a denial of service attack by doing a bunch of connects
-//       // really fast and see what that does to the server's fds.
-//       getLogWriter().info("doing denial of service attach");
-//       vm0.invoke(denialOfService);
-//       // @todo darrel: check fd limit?
-    }
-    finally {
-      stopBridgeServers(getCache());
-    }
-    }
-    finally {
-      System.getProperties().remove(AcceptorImpl.ACCEPT_TIMEOUT_PROPERTY_NAME);
-    }
-  }
-
-  public void testSynchronousEvents() throws Exception {
-    InternalBridgeMembership.setForceSynchronous(true);
-    try {
-      doTestBasicEvents();
-    }
-    finally {
-      InternalBridgeMembership.setForceSynchronous(false);
-    }
-  }
-  
-  /**
-   * Tests event notification methods on BridgeMembership.
-   */
-  public void testBasicEvents() throws Exception {
-    doTestBasicEvents();
-  }
-  
-  public void doTestBasicEvents() throws Exception {
-    final boolean[] fired = new boolean[3];
-    final DistributedMember[] member = new DistributedMember[3];
-    final String[] memberId = new String[3];
-    final boolean[] isClient = new boolean[3];
-    
-    BridgeMembershipListener listener = new BridgeMembershipListener() {
-      public synchronized void memberJoined(BridgeMembershipEvent event) {
-        fired[JOINED] = true;
-        member[JOINED] = event.getMember();
-        memberId[JOINED] = event.getMemberId();
-        isClient[JOINED] = event.isClient();
-        notify();
-      }
-      public synchronized void memberLeft(BridgeMembershipEvent event) {
-        fired[LEFT] = true;
-        member[LEFT] = event.getMember();
-        memberId[LEFT] = event.getMemberId();
-        isClient[LEFT] = event.isClient();
-        notify();
-      }
-      public synchronized void memberCrashed(BridgeMembershipEvent event) {
-        fired[CRASHED] = true;
-        member[CRASHED] = event.getMember();
-        memberId[CRASHED] = event.getMemberId();
-        isClient[CRASHED] = event.isClient();
-        notify();
-      }
-    };
-    BridgeMembership.registerBridgeMembershipListener(listener);
-    
-    // test JOIN for server
-    DistributedMember serverJoined = new TestDistributedMember("serverJoined");
-    InternalBridgeMembership.notifyJoined(serverJoined, SERVER);
-    synchronized(listener) {
-      if (!fired[JOINED]) {
-        listener.wait(2000);
-      }
-    }
-    assertTrue(fired[JOINED]);
-    assertEquals(serverJoined, member[JOINED]);
-    assertEquals(serverJoined.getId(), memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    // test JOIN for client
-    DistributedMember clientJoined = new TestDistributedMember("clientJoined");
-    InternalBridgeMembership.notifyJoined(clientJoined, CLIENT);
-    synchronized(listener) {
-      if (!fired[JOINED]) {
-        listener.wait(2000);
-      }
-    }
-    assertTrue(fired[JOINED]);
-    assertEquals(clientJoined, member[JOINED]);
-    assertEquals(clientJoined.getId(), memberId[JOINED]);
-    assertTrue(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    // test LEFT for server
-    DistributedMember serverLeft = new TestDistributedMember("serverLeft");
-    InternalBridgeMembership.notifyLeft(serverLeft, SERVER);
-    synchronized(listener) {
-      if (!fired[LEFT]) {
-        listener.wait(2000);
-      }
-    }
-    assertFalse(fired[JOINED]);
-    assertNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertTrue(fired[LEFT]);
-    assertEquals(serverLeft, member[LEFT]);
-    assertEquals(serverLeft.getId(), memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    // test LEFT for client
-    DistributedMember clientLeft = new TestDistributedMember("clientLeft");
-    InternalBridgeMembership.notifyLeft(clientLeft, CLIENT);
-    synchronized(listener) {
-      if (!fired[LEFT]) {
-        listener.wait(2000);
-      }
-    }
-    assertFalse(fired[JOINED]);
-    assertNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertTrue(fired[LEFT]);
-    assertEquals(clientLeft, member[LEFT]);
-    assertEquals(clientLeft.getId(), memberId[LEFT]);
-    assertTrue(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    // test CRASHED for server
-    DistributedMember serverCrashed = new TestDistributedMember("serverCrashed");
-    InternalBridgeMembership.notifyCrashed(serverCrashed, SERVER);
-    synchronized(listener) {
-      if (!fired[CRASHED]) {
-        listener.wait(2000);
-      }
-    }
-    assertFalse(fired[JOINED]);
-    assertNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertTrue(fired[CRASHED]);
-    assertEquals(serverCrashed, member[CRASHED]);
-    assertEquals(serverCrashed.getId(), memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    // test CRASHED for client
-    DistributedMember clientCrashed = new TestDistributedMember("clientCrashed");
-    InternalBridgeMembership.notifyCrashed(clientCrashed, CLIENT);
-    synchronized(listener) {
-      if (!fired[CRASHED]) {
-        listener.wait(2000);
-      }
-    }
-    assertFalse(fired[JOINED]);
-    assertNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertTrue(fired[CRASHED]);
-    assertEquals(clientCrashed, member[CRASHED]);
-    assertEquals(clientCrashed.getId(), memberId[CRASHED]);
-    assertTrue(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-  }
-  
-  /**
-   * Resets all elements of arrays used for listener testing. Boolean values
-   * are reset to false. String values are reset to null.
-   */
-  private void resetArraysForTesting(boolean[] fired, 
-                                     DistributedMember[] member,
-                                     String[] memberId, 
-                                     boolean[] isClient) {
-    for (int i = 0; i < fired.length; i++) {
-      fired[i] = false;
-      member[i] = null;
-      memberId[i] = null;
-      isClient[i] = false;
-    }
-  }
-  
-  /**
-   * Tests unregisterBridgeMembershipListener to ensure that no further events
-   * are delivered to unregistered listeners.
-   */
-  public void testUnregisterBridgeMembershipListener() throws Exception {
-    final boolean[] fired = new boolean[1];
-    final DistributedMember[] member = new DistributedMember[1];
-    final String[] memberId = new String[1];
-    final boolean[] isClient = new boolean[1];
-    
-    BridgeMembershipListener listener = new BridgeMembershipListener() {
-      public synchronized void memberJoined(BridgeMembershipEvent event) {
-        fired[0] = true;
-        member[0] = event.getMember();
-        memberId[0] = event.getMemberId();
-        isClient[0] = event.isClient();
-        notify();
-      }
-      public void memberLeft(BridgeMembershipEvent event) {
-      }
-      public void memberCrashed(BridgeMembershipEvent event) {
-      }
-    };
-    BridgeMembership.registerBridgeMembershipListener(listener);
-    
-    // fire event to make sure listener is registered
-    DistributedMember clientJoined = new TestDistributedMember("clientJoined");
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    synchronized(listener) {
-      if (!fired[0]) {
-        listener.wait(2000);
-      }
-    }
-    assertTrue(fired[0]);
-    assertEquals(clientJoined, member[0]);
-    assertEquals(clientJoined.getId(), memberId[0]);
-    assertTrue(isClient[0]);
-
-    resetArraysForTesting(fired, member, memberId, isClient);
-    assertFalse(fired[0]);
-    assertNull(memberId[0]);
-    assertFalse(isClient[0]);
-
-    // unregister and verify listener is not notified
-    BridgeMembership.unregisterBridgeMembershipListener(listener);
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    synchronized(listener) {
-      listener.wait(20);
-    }
-    assertFalse(fired[0]);
-    assertNull(member[0]);
-    assertNull(memberId[0]);
-    assertFalse(isClient[0]);
-  }
-  
-  public void testMultipleListeners() throws Exception {
-    final int NUM_LISTENERS = 4;
-    final boolean[] fired = new boolean[NUM_LISTENERS];
-    final DistributedMember[] member = new DistributedMember[NUM_LISTENERS];
-    final String[] memberId = new String[NUM_LISTENERS];
-    final boolean[] isClient = new boolean[NUM_LISTENERS];
-    
-    final BridgeMembershipListener[] listeners = new BridgeMembershipListener[NUM_LISTENERS];
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      final int whichListener = i;
-      listeners[i] = new BridgeMembershipListener() {
-        public synchronized void memberJoined(BridgeMembershipEvent event) {
-          assertFalse(fired[whichListener]);
-          assertNull(member[whichListener]);
-          assertNull(memberId[whichListener]);
-          assertFalse(isClient[whichListener]);
-          fired[whichListener] = true;
-          member[whichListener] = event.getMember();
-          memberId[whichListener] = event.getMemberId();
-          isClient[whichListener] = event.isClient();
-          notify();
-        }
-        public void memberLeft(BridgeMembershipEvent event) {
-        }
-        public void memberCrashed(BridgeMembershipEvent event) {
-        }
-      };
-    }
-    
-    final DistributedMember clientJoined = new TestDistributedMember("clientJoined");
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      synchronized(listeners[i]) {
-        listeners[i].wait(20);
-      }
-      assertFalse(fired[i]);
-      assertNull(member[i]);
-      assertNull(memberId[i]);
-      assertFalse(isClient[i]);
-    }
-    
-    // attempt to register same listener twice... 2nd reg should be ignored
-    // failure would cause an assertion failure in memberJoined impl
-    BridgeMembership.registerBridgeMembershipListener(listeners[0]);
-    BridgeMembership.registerBridgeMembershipListener(listeners[0]);
-    
-    BridgeMembershipListener[] registeredListeners = 
-      BridgeMembership.getBridgeMembershipListeners();
-    assertEquals(1, registeredListeners.length);
-    assertEquals(listeners[0], registeredListeners[0]);
-    
-    BridgeMembership.registerBridgeMembershipListener(listeners[1]);
-    registeredListeners = BridgeMembership.getBridgeMembershipListeners();
-    assertEquals(2, registeredListeners.length);
-    assertEquals(listeners[0], registeredListeners[0]);
-    assertEquals(listeners[1], registeredListeners[1]);
-
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    synchronized(listeners[1]) {
-      if (!fired[1]) {
-        listeners[1].wait(2000);
-      }
-    }
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      if (i < 2) {
-        assertTrue(fired[i]);
-        assertEquals(clientJoined, member[i]);
-        assertEquals(clientJoined.getId(), memberId[i]);
-        assertTrue(isClient[i]);
-      } else {
-        assertFalse(fired[i]);
-        assertNull(member[i]);
-        assertNull(memberId[i]);
-        assertFalse(isClient[i]);
-      }
-    }
-    resetArraysForTesting(fired, member, memberId, isClient);
-        
-    BridgeMembership.unregisterBridgeMembershipListener(listeners[0]);
-    registeredListeners = BridgeMembership.getBridgeMembershipListeners();
-    assertEquals(1, registeredListeners.length);
-    assertEquals(listeners[1], registeredListeners[0]);
-    
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    synchronized(listeners[1]) {
-      if (!fired[1]) {
-        listeners[1].wait(2000);
-      }
-    }
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      if (i == 1) {
-        assertTrue(fired[i]);
-        assertEquals(clientJoined, member[i]);
-        assertEquals(clientJoined.getId(), memberId[i]);
-        assertTrue(isClient[i]);
-      } else {
-        assertFalse(fired[i]);
-        assertNull(member[i]);
-        assertNull(memberId[i]);
-        assertFalse(isClient[i]);
-      }
-    }
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    BridgeMembership.registerBridgeMembershipListener(listeners[2]);
-    BridgeMembership.registerBridgeMembershipListener(listeners[3]);
-    registeredListeners = BridgeMembership.getBridgeMembershipListeners();
-    assertEquals(3, registeredListeners.length);
-    assertEquals(listeners[1], registeredListeners[0]);
-    assertEquals(listeners[2], registeredListeners[1]);
-    assertEquals(listeners[3], registeredListeners[2]);
-
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    synchronized(listeners[3]) {
-      if (!fired[3]) {
-        listeners[3].wait(2000);
-      }
-    }
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      if (i != 0) {
-        assertTrue(fired[i]);
-        assertEquals(clientJoined, member[i]);
-        assertEquals(clientJoined.getId(), memberId[i]);
-        assertTrue(isClient[i]);
-      } else {
-        assertFalse(fired[i]);
-        assertNull(member[i]);
-        assertNull(memberId[i]);
-        assertFalse(isClient[i]);
-      }
-    }
-    resetArraysForTesting(fired, member, memberId, isClient);
-    
-    BridgeMembership.registerBridgeMembershipListener(listeners[0]);
-    registeredListeners = BridgeMembership.getBridgeMembershipListeners();
-    assertEquals(4, registeredListeners.length);
-    assertEquals(listeners[1], registeredListeners[0]);
-    assertEquals(listeners[2], registeredListeners[1]);
-    assertEquals(listeners[3], registeredListeners[2]);
-    assertEquals(listeners[0], registeredListeners[3]);
-
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    synchronized(listeners[0]) {
-      if (!fired[0]) {
-        listeners[0].wait(2000);
-      }
-    }
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      assertTrue(fired[i]);
-      assertEquals(clientJoined, member[i]);
-      assertEquals(clientJoined.getId(), memberId[i]);
-      assertTrue(isClient[i]);
-    }
-    resetArraysForTesting(fired, member, memberId, isClient);
-    
-    BridgeMembership.unregisterBridgeMembershipListener(listeners[3]);
-    registeredListeners = BridgeMembership.getBridgeMembershipListeners();
-    assertEquals(3, registeredListeners.length);
-    assertEquals(listeners[1], registeredListeners[0]);
-    assertEquals(listeners[2], registeredListeners[1]);
-    assertEquals(listeners[0], registeredListeners[2]);
-    
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    synchronized(listeners[0]) {
-      if (!fired[0]) {
-        listeners[0].wait(2000);
-      }
-    }
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      if (i < 3) {
-        assertTrue(fired[i]);
-        assertEquals(clientJoined, member[i]);
-        assertEquals(clientJoined.getId(), memberId[i]);
-        assertTrue(isClient[i]);
-      } else {
-        assertFalse(fired[i]);
-        assertNull(member[i]);
-        assertNull(memberId[i]);
-        assertFalse(isClient[i]);
-      }
-    }
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    BridgeMembership.unregisterBridgeMembershipListener(listeners[2]);
-    registeredListeners = BridgeMembership.getBridgeMembershipListeners();
-    assertEquals(2, registeredListeners.length);
-    assertEquals(listeners[1], registeredListeners[0]);
-    assertEquals(listeners[0], registeredListeners[1]);
-    
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    synchronized(listeners[0]) {
-      if (!fired[0]) {
-        listeners[0].wait(2000);
-      }
-    }
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      if (i < 2) {
-        assertTrue(fired[i]);
-        assertEquals(clientJoined, member[i]);
-        assertEquals(clientJoined.getId(), memberId[i]);
-        assertTrue(isClient[i]);
-      } else {
-        assertFalse(fired[i]);
-        assertNull(member[i]);
-        assertNull(memberId[i]);
-        assertFalse(isClient[i]);
-      }
-    }
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    BridgeMembership.unregisterBridgeMembershipListener(listeners[1]);
-    BridgeMembership.unregisterBridgeMembershipListener(listeners[0]);
-    registeredListeners = BridgeMembership.getBridgeMembershipListeners();
-    assertEquals(0, registeredListeners.length);
-    
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      synchronized(listeners[i]) {
-        listeners[i].wait(20);
-      }
-      assertFalse(fired[i]);
-      assertNull(member[i]);
-      assertNull(memberId[i]);
-      assertFalse(isClient[i]);
-    }
-    resetArraysForTesting(fired, member, memberId, isClient);
-    
-    BridgeMembership.registerBridgeMembershipListener(listeners[1]);
-    registeredListeners = BridgeMembership.getBridgeMembershipListeners();
-    assertEquals(1, registeredListeners.length);
-    assertEquals(listeners[1], registeredListeners[0]);
-    
-    InternalBridgeMembership.notifyJoined(clientJoined, true);
-    synchronized(listeners[1]) {
-      if (!fired[1]) {
-        listeners[1].wait(2000);
-      }
-    }
-    for (int i = 0; i < NUM_LISTENERS; i++) {
-      if (i == 1) {
-        assertTrue(fired[i]);
-        assertEquals(clientJoined, member[i]);
-        assertEquals(clientJoined.getId(), memberId[i]);
-        assertTrue(isClient[i]);
-      } else {
-        assertFalse(fired[i]);
-        assertNull(member[i]);
-        assertNull(memberId[i]);
-        assertFalse(isClient[i]);
-      }
-    }
-  }
- 
-  protected static int testBridgeMembershipEventsInClient_port;
-  private static int getTestBridgeMembershipEventsInClient_port() {
-    return testBridgeMembershipEventsInClient_port;
-  }
-  /**
-   * Tests notification of events in client process. Bridge clients detect
-   * server joins when the client connects to the server. If the server
-   * crashes or departs gracefully, the client will detect this as a crash.
-   */
-  public void testBridgeMembershipEventsInClient() throws Exception {
-    addExpectedException("IOException");
-    final boolean[] fired = new boolean[3];
-    final DistributedMember[] member = new DistributedMember[3];
-    final String[] memberId = new String[3];
-    final boolean[] isClient = new boolean[3];
-    
-    // create and register BridgeMembershipListener in controller vm...
-    BridgeMembershipListener listener = new BridgeMembershipListener() {
-      public synchronized void memberJoined(BridgeMembershipEvent event) {
-        getLogWriter().info("[testBridgeMembershipEventsInClient] memberJoined: " + event);
-        fired[JOINED] = true;
-        member[JOINED] = event.getMember();
-        memberId[JOINED] = event.getMemberId();
-        isClient[JOINED] = event.isClient();
-        notifyAll();
-      }
-      public synchronized void memberLeft(BridgeMembershipEvent event) {
-        getLogWriter().info("[testBridgeMembershipEventsInClient] memberLeft: " + event);
-//        fail("Please update testBridgeMembershipEventsInClient to handle memberLeft for BridgeServer.");
-      }
-      public synchronized void memberCrashed(BridgeMembershipEvent event) {
-        getLogWriter().info("[testBridgeMembershipEventsInClient] memberCrashed: " + event);
-        fired[CRASHED] = true;
-        member[CRASHED] = event.getMember();
-        memberId[CRASHED] = event.getMemberId();
-        isClient[CRASHED] = event.isClient();
-        notifyAll();
-      }
-    };
-    BridgeMembership.registerBridgeMembershipListener(listener);
-
-    final VM vm0 = Host.getHost(0).getVM(0);
-    final String name = this.getUniqueName();
-    final int[] ports = new int[1];
-
-    // create BridgeServer in vm0...
-    vm0.invoke(new CacheSerializableRunnable("Create BridgeServer") {
-      public void run2() throws CacheException {
-        try {
-          getLogWriter().info("[testBridgeMembershipEventsInClient] Create BridgeServer");
-          getSystem();
-          AttributesFactory factory = new AttributesFactory();
-          factory.setScope(Scope.LOCAL);
-          Region region = createRegion(name, factory.create());
-          assertNotNull(region);
-          assertNotNull(getRootRegion().getSubregion(name));
-          testBridgeMembershipEventsInClient_port = startBridgeServer(0);
-        }
-        catch(IOException e) {
-          getSystem().getLogWriter().fine(new Exception(e));
-          fail("Failed to start CacheServer on VM1: " + e.getMessage());
-        }
-      }
-    });
-    
-    // gather details for later creation of ConnectionPool...
-    ports[0] = vm0.invokeInt(BridgeMembershipDUnitTest.class, 
-                             "getTestBridgeMembershipEventsInClient_port");
-    assertTrue(ports[0] != 0);
-
-    DistributedMember serverMember = (DistributedMember) vm0.invoke(BridgeMembershipDUnitTest.class,
-    "getDistributedMember");
-
-    String serverMemberId = (String) vm0.invoke(BridgeMembershipDUnitTest.class,
-                                                "getMemberId");
-
-    getLogWriter().info("[testBridgeMembershipEventsInClient] ports[0]=" + ports[0]);
-    getLogWriter().info("[testBridgeMembershipEventsInClient] serverMember=" + serverMember);
-    getLogWriter().info("[testBridgeMembershipEventsInClient] serverMemberId=" + serverMemberId);
-
-    assertFalse(fired[JOINED]);
-    assertNull(member[JOINED]);
-    assertNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(member[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(member[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    
-    // sanity check...
-    getLogWriter().info("[testBridgeMembershipEventsInClient] sanity check");
-    DistributedMember test = new TestDistributedMember("test");
-    InternalBridgeMembership.notifyJoined(test, SERVER);
-    synchronized(listener) {
-      if (!fired[JOINED] && !fired[CRASHED]) {
-        listener.wait(2000);
-      }
-    }
-    
-    assertTrue(fired[JOINED]);
-    assertEquals(test, member[JOINED]);
-    assertEquals(test.getId(), memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(member[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(member[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-    
-    // create bridge client in controller vm...
-    getLogWriter().info("[testBridgeMembershipEventsInClient] create bridge client");
-    Properties config = new Properties();
-    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    getSystem(config);
-    
-    try {
-      getCache();
-      AttributesFactory factory = new AttributesFactory();
-      factory.setScope(Scope.LOCAL);
-      BridgeTestCase.configureConnectionPool(factory, getServerHostName(Host.getHost(0)), ports, true, -1, -1, null);
-      createRegion(name, factory.create());
-      assertNotNull(getRootRegion().getSubregion(name));
-    }
-    catch (CacheException ex) {
-      fail("While creating Region on Edge", ex);
-    }
-    synchronized(listener) {
-      if (!fired[JOINED] && !fired[CRASHED]) {
-        listener.wait(60 * 1000);
-      }
-    }
-    
-    getLogWriter().info("[testBridgeMembershipEventsInClient] assert client detected server join");
-    
-    // first check the getCurrentServers() result
-    ClientCache clientCache = (ClientCache)getCache();
-    Set<InetSocketAddress> servers = clientCache.getCurrentServers();
-    assertTrue(!servers.isEmpty());
-    InetSocketAddress serverAddr = servers.iterator().next();
-    InetSocketAddress expectedAddr = new InetSocketAddress(serverMember.getHost(), ports[0]);
-    assertEquals(expectedAddr, serverAddr);
-    
-    // now check listener results
-    assertTrue(fired[JOINED]);
-    assertNotNull(member[JOINED]);
-    assertNotNull(memberId[JOINED]);
-    assertEquals(serverMember, member[JOINED]);
-    assertEquals(serverMemberId, memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(member[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(member[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    vm0.invoke(new SerializableRunnable("Stop BridgeServer") {
-      public void run() {
-        getLogWriter().info("[testBridgeMembershipEventsInClient] Stop BridgeServer");
-        stopBridgeServers(getCache());
-      }
-    });
-    synchronized(listener) {
-      if (!fired[JOINED] && !fired[CRASHED]) {
-        listener.wait(60 * 1000);
-      }
-    }
-    
-    getLogWriter().info("[testBridgeMembershipEventsInClient] assert client detected server departure");
-    assertFalse(fired[JOINED]);
-    assertNull(member[JOINED]);
-    assertNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(member[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertTrue(fired[CRASHED]);
-    assertNotNull(member[CRASHED]);
-    assertNotNull(memberId[CRASHED]);
-    assertEquals(serverMember, member[CRASHED]);
-    assertEquals(serverMemberId, memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-    
-    //now test that we redisover the bridge server
-    vm0.invoke(new CacheSerializableRunnable("Recreate BridgeServer") {
-      public void run2() throws CacheException {
-        try {
-          getLogWriter().info("[testBridgeMembershipEventsInClient] restarting BridgeServer");
-          startBridgeServer(ports[0]);
-        }
-        catch(IOException e) {
-          getSystem().getLogWriter().fine(new Exception(e));
-          fail("Failed to start CacheServer on VM1: " + e.getMessage());
-        }
-      }
-    });
-    synchronized(listener) {
-      if (!fired[JOINED] && !fired[CRASHED]) {
-        listener.wait(60 * 1000);
-      }
-    }
-    
-    getLogWriter().info("[testBridgeMembershipEventsInClient] assert client detected server recovery");
-    assertTrue(fired[JOINED]);
-    assertNotNull(member[JOINED]);
-    assertNotNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertEquals(serverMember, member[JOINED]);
-    assertEquals(serverMemberId, memberId[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(member[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(member[CRASHED]);
-    assertNull(memberId[CRASHED]);
-  }
-  
-  /**
-   * Tests notification of events in server process. Bridge servers detect
-   * client joins when the client connects to the server.
-   */
-  public void testBridgeMembershipEventsInServer() throws Exception {
-    final boolean[] fired = new boolean[3];
-    final DistributedMember[] member = new DistributedMember[3];
-    final String[] memberId = new String[3];
-    final boolean[] isClient = new boolean[3];
-    
-    // create and register BridgeMembershipListener in controller vm...
-    BridgeMembershipListener listener = new BridgeMembershipListener() {
-      public synchronized void memberJoined(BridgeMembershipEvent event) {
-        getLogWriter().info("[testBridgeMembershipEventsInServer] memberJoined: " + event);
-        fired[JOINED] = true;
-        member[JOINED] = event.getMember();
-        memberId[JOINED] = event.getMemberId();
-        isClient[JOINED] = event.isClient();
-        notifyAll();
-        assertFalse(fired[LEFT] || fired[CRASHED]);
-      }
-      public synchronized void memberLeft(BridgeMembershipEvent event) {
-        getLogWriter().info("[testBridgeMembershipEventsInServer] memberLeft: " + event);
-        fired[LEFT] = true;
-        member[LEFT] = event.getMember();
-        memberId[LEFT] = event.getMemberId();
-        isClient[LEFT] = event.isClient();
-        notifyAll();
-        assertFalse(fired[JOINED] || fired[CRASHED]);
-      }
-      public synchronized void memberCrashed(BridgeMembershipEvent event) {
-        getLogWriter().info("[testBridgeMembershipEventsInServer] memberCrashed: " + event);
-        fired[CRASHED] = true;
-        member[CRASHED] = event.getMember();
-        memberId[CRASHED] = event.getMemberId();
-        isClient[CRASHED] = event.isClient();
-        notifyAll();
-        assertFalse(fired[JOINED] || fired[LEFT]);
-      }
-    };
-    BridgeMembership.registerBridgeMembershipListener(listener);
-
-    final VM vm0 = Host.getHost(0).getVM(0);
-    final String name = this.getUniqueName();
-    final int[] ports = new int[1];
-
-    // create BridgeServer in controller vm...
-    getLogWriter().info("[testBridgeMembershipEventsInServer] Create BridgeServer");
-    getSystem();
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    Region region = createRegion(name, factory.create());
-    assertNotNull(region);
-    assertNotNull(getRootRegion().getSubregion(name));
-    
-    ports[0] = startBridgeServer(0);
-    assertTrue(ports[0] != 0);
-    String serverMemberId = getMemberId();
-    DistributedMember serverMember = getDistributedMember();
-
-    getLogWriter().info("[testBridgeMembershipEventsInServer] ports[0]=" + ports[0]);
-    getLogWriter().info("[testBridgeMembershipEventsInServer] serverMemberId=" + serverMemberId);
-    getLogWriter().info("[testBridgeMembershipEventsInServer] serverMember=" + serverMember);
-
-    assertFalse(fired[JOINED]);
-    assertNull(member[JOINED]);
-    assertNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(member[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(member[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    
-    // sanity check...
-    getLogWriter().info("[testBridgeMembershipEventsInServer] sanity check");
-    DistributedMember test = new TestDistributedMember("test");
-    InternalBridgeMembership.notifyJoined(test, CLIENT);
-    synchronized(listener) {
-      if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
-        listener.wait(2000);
-      }
-    }
-    assertTrue(fired[JOINED]);
-    assertEquals(test, member[JOINED]);
-    assertEquals(test.getId(), memberId[JOINED]);
-    assertTrue(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(member[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(member[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-    
-    final Host host = Host.getHost(0);
-    SerializableRunnable createConnectionPool =
-    new CacheSerializableRunnable("Create connectionPool") {
-      public void run2() throws CacheException {
-        getLogWriter().info("[testBridgeMembershipEventsInServer] create bridge client");
-        Properties config = new Properties();
-        config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-        config.setProperty(DistributionConfig.LOCATORS_NAME, "");
-        getSystem(config);
-        AttributesFactory factory = new AttributesFactory();
-        factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, 2, null);
-        createRegion(name, factory.create());
-        assertNotNull(getRootRegion().getSubregion(name));
-      }
-    };
-
-    // create bridge client in vm0...
-    vm0.invoke(createConnectionPool);
-    String clientMemberId = (String) vm0.invoke(BridgeMembershipDUnitTest.class,
-                                                "getMemberId");
-    DistributedMember clientMember = (DistributedMember) vm0.invoke(BridgeMembershipDUnitTest.class,
-                                                "getDistributedMember");
-                                                
-    synchronized(listener) {
-      if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
-        listener.wait(60000);
-      }
-    }
-    
-    getLogWriter().info("[testBridgeMembershipEventsInServer] assert server detected client join");
-    assertTrue(fired[JOINED]);
-    assertEquals(member[JOINED] + " should equal " + clientMember,
-      clientMember, member[JOINED]);
-    assertEquals(memberId[JOINED] + " should equal " + clientMemberId,
-      clientMemberId, memberId[JOINED]);
-    assertTrue(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(member[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(member[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    pauseForClientToJoin();
-    
-    vm0.invoke(new SerializableRunnable("Stop bridge client") {
-      public void run() {
-        getLogWriter().info("[testBridgeMembershipEventsInServer] Stop bridge client");
-        getRootRegion().getSubregion(name).close();
-        Map m = PoolManager.getAll();
-        Iterator mit = m.values().iterator();
-        while(mit.hasNext()) {
-          Pool p = (Pool)mit.next();
-          p.destroy();
-        }
-      }
-    });
-
-    synchronized(listener) {
-      if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
-        listener.wait(60000);
-      }
-    }
-    
-    getLogWriter().info("[testBridgeMembershipEventsInServer] assert server detected client left");
-    assertFalse(fired[JOINED]);
-    assertNull(member[JOINED]);
-    assertNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    assertTrue(fired[LEFT]);
-    assertEquals(clientMember, member[LEFT]);
-    assertEquals(clientMemberId, memberId[LEFT]);
-    assertTrue(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(member[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-
-    // reconnect bridge client to test for crashed event
-    vm0.invoke(createConnectionPool);
-    clientMemberId = (String) vm0.invoke(BridgeMembershipDUnitTest.class,
-                                         "getMemberId");
-                                                
-    synchronized(listener) {
-      if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
-        listener.wait(60000);
-      }
-    }
-    
-    getLogWriter().info("[testBridgeMembershipEventsInServer] assert server detected client re-join");
-    assertTrue(fired[JOINED]);
-    assertEquals(clientMember, member[JOINED]);
-    assertEquals(clientMemberId, memberId[JOINED]);
-    assertTrue(isClient[JOINED]);
-    assertFalse(fired[LEFT]);
-    assertNull(member[LEFT]);
-    assertNull(memberId[LEFT]);
-    assertFalse(isClient[LEFT]);
-    assertFalse(fired[CRASHED]);
-    assertNull(member[CRASHED]);
-    assertNull(memberId[CRASHED]);
-    assertFalse(isClient[CRASHED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-    
-    pauseForClientToJoin();
-
-    ServerConnection.setForceClientCrashEvent(true);
-    try {
-      vm0.invoke(new SerializableRunnable("Stop bridge client") {
-        public void run() {
-          getLogWriter().info("[testBridgeMembershipEventsInServer] Stop bridge client");
-          getRootRegion().getSubregion(name).close();
-          Map m = PoolManager.getAll();
-          Iterator mit = m.values().iterator();
-          while(mit.hasNext()) {
-            Pool p = (Pool)mit.next();
-            p.destroy();
-          }
-        }
-      });
-  
-      synchronized(listener) {
-        if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
-          listener.wait(60000);
-        }
-      }
-      
-      getLogWriter().info("[testBridgeMembershipEventsInServer] assert server detected client crashed");
-      assertFalse(fired[JOINED]);
-      assertNull(member[JOINED]);
-      assertNull(memberId[JOINED]);
-      assertFalse(isClient[JOINED]);
-      assertFalse(fired[LEFT]);
-      assertNull(member[LEFT]);
-      assertNull(memberId[LEFT]);
-      assertFalse(isClient[LEFT]);
-      assertTrue(fired[CRASHED]);
-      assertEquals(clientMember, member[CRASHED]);
-      assertEquals(clientMemberId, memberId[CRASHED]);
-      assertTrue(isClient[CRASHED]);
-    }
-    finally {
-      ServerConnection.setForceClientCrashEvent(false);
-    }
-  }
-  
-  /**
-   * The joined event fires when the first client handshake is processed.
-   * This pauses long enough to allow the rest of the client sockets to
-   * complete handshaking before making the client leave. Without doing this
-   * subsequent socket handshakes that are processed could fire join events
-   * after departure events and then a departure event again. If you see
-   * failures in testBridgeMembershipEventsInServer, try increasing this
-   * timeout.
-   */
-  private void pauseForClientToJoin() {
-    pause(2000);
-  }
-  
-  /** 
-   * Tests registration and event notification in conjunction with 
-   * disconnecting and reconnecting to DistributedSystem. 
-   */
-  public void testLifecycle() throws Exception {
-    final boolean[] fired = new boolean[3];
-    final DistributedMember[] member = new DistributedMember[3];
-    final String[] memberId = new String[3];
-    final boolean[] isClient = new boolean[3];
-    
-    // create and register BridgeMembershipListener in controller vm...
-    BridgeMembershipListener listener = new BridgeMembershipListener() {
-      public synchronized void memberJoined(BridgeMembershipEvent event) {
-        assertFalse(fired[JOINED]);
-        assertNull(member[JOINED]);
-        assertNull(memberId[JOINED]);
-        assertFalse(isClient[JOINED]);
-        fired[JOINED] = true;
-        member[JOINED] = event.getMember();
-        memberId[JOINED] = event.getMemberId();
-        isClient[JOINED] = event.isClient();
-        notifyAll();
-      }
-      public synchronized void memberLeft(BridgeMembershipEvent event) {
-      }
-      public synchronized void memberCrashed(BridgeMembershipEvent event) {
-      }
-    };
-    BridgeMembership.registerBridgeMembershipListener(listener);
-    
-    // create loner in controller vm...
-    Properties config = new Properties();
-    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    getSystem(config);
-    
-    // assert that event is fired while connected
-    DistributedMember serverJoined = new TestDistributedMember("serverJoined");
-    InternalBridgeMembership.notifyJoined(serverJoined, SERVER);
-    synchronized(listener) {
-      if (!fired[JOINED]) {
-        listener.wait(2000);
-      }
-    }
-    assertTrue(fired[JOINED]);
-    assertEquals(serverJoined, member[JOINED]);
-    assertEquals(serverJoined.getId(), memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-    
-    // assert that event is NOT fired while disconnected
-    disconnectFromDS();
-    
-
-    InternalBridgeMembership.notifyJoined(serverJoined, SERVER);
-    synchronized(listener) {
-      listener.wait(20);
-    }
-    assertFalse(fired[JOINED]);
-    assertNull(member[JOINED]);
-    assertNull(memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-    resetArraysForTesting(fired, member, memberId, isClient);
-    
-    // assert that event is fired again after reconnecting
-    InternalDistributedSystem sys = getSystem(config);
-    assertTrue(sys.isConnected());
-
-    InternalBridgeMembership.notifyJoined(serverJoined, SERVER);
-    synchronized(listener) {
-      if (!fired[JOINED]) {
-        listener.wait(2000);
-      }
-    }
-    assertTrue(fired[JOINED]);
-    assertEquals(serverJoined, member[JOINED]);
-    assertEquals(serverJoined.getId(), memberId[JOINED]);
-    assertFalse(isClient[JOINED]);
-  }
-  
-  /**
-   * Starts up server in controller vm and 4 clients, then calls and tests
-   * BridgeMembership.getConnectedClients(). 
-   */
-  public void testGetConnectedClients() throws Exception {
-    final String name = this.getUniqueName();
-    final int[] ports = new int[1];
-
-    // create BridgeServer in controller vm...
-    getLogWriter().info("[testGetConnectedClients] Create BridgeServer");
-    getSystem();
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-    Region region = createRegion(name, factory.create());
-    assertNotNull(region);
-    assertNotNull(getRootRegion().getSubregion(name));
-    
-    ports[0] = startBridgeServer(0);
-    assertTrue(ports[0] != 0);
-    String serverMemberId = getMemberId();
-
-    getLogWriter().info("[testGetConnectedClients] ports[0]=" + ports[0]);
-    getLogWriter().info("[testGetConnectedClients] serverMemberId=" + serverMemberId);
-
-    final Host host = Host.getHost(0);
-    SerializableRunnable createPool =
-    new CacheSerializableRunnable("Create connection pool") {
-      public void run2() throws CacheException {
-        getLogWriter().info("[testGetConnectedClients] create bridge client");
-        Properties config = new Properties();
-        config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-        config.setProperty(DistributionConfig.LOCATORS_NAME, "");
-        getSystem(config);
-        AttributesFactory factory = new AttributesFactory();
-        factory.setScope(Scope.LOCAL);
-        BridgeTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, -1, null);
-        createRegion(name, factory.create());
-        assertNotNull(getRootRegion().getSubregion(name));
-      }
-    };
-
-    // create bridge client in vm0...
-    final String[] clientMemberIdArray = new String[host.getVMCount()];
-    
-    for (int i = 0; i < host.getVMCount(); i++) { 
-      final VM vm = Host.getHost(0).getVM(i);
-      vm.invoke(createPool);
-      clientMemberIdArray[i] =  String.valueOf(vm.invoke(
-        BridgeMembershipDUnitTest.class, "getMemberId"));
-    }
-    Collection clientMemberIds = Arrays.asList(clientMemberIdArray);
-                                                
-    {
-      final int expectedClientCount = clientMemberIds.size();
-      WaitCriterion wc = new WaitCriterion() {
-        public String description() {
-          return "wait for clients";
-        }
-        public boolean done() {
-          Map connectedClients = InternalBridgeMembership.getConnectedClients(false);
-          if (connectedClients == null) {
-            return false;
-          }
-          if (connectedClients.size() != expectedClientCount) {
-            return false;
-          }
-          return true;
-        }
-      };
-      waitForCriterion(wc, 10000, 100, false);
-    }
-    
-    Map connectedClients = InternalBridgeMembership.getConnectedClients(false);
-    assertNotNull(connectedClients);
-    assertEquals(clientMemberIds.size(), connectedClients.size());
-    for (Iterator iter = connectedClients.keySet().iterator(); iter.hasNext();) {
-      String connectedClient = (String)iter.next();
-      getLogWriter().info("[testGetConnectedClients] checking for client " + connectedClient);
-      assertTrue(clientMemberIds.contains(connectedClient));
-      getLogWriter().info("[testGetConnectedClients] count for connectedClient: " + 
-                          connectedClients.get(connectedClient));
-    }
-  }
-
-  /**
-   * Starts up 4 server and the controller vm as a client, then calls and tests
-   * BridgeMembership.getConnectedServers(). 
-   */
-  public void testGetConnectedServers() throws Exception {
-    final Host host = Host.getHost(0);
-    final String name = this.getUniqueName();
-    final int[] ports = new int[host.getVMCount()];
-    
-    for (int i = 0; i < host.getVMCount(); i++) { 
-      final int whichVM = i;
-      final VM vm = Host.getHost(0).getVM(i);
-      vm.invoke(new CacheSerializableRunnable("Create bridge server") {
-        public void run2() throws CacheException {
-          // create BridgeServer in controller vm...
-          getLogWriter().info("[testGetConnectedServers] Create BridgeServer");
-          getSystem();
-          AttributesFactory factory = new AttributesFactory();
-          factory.setScope(Scope.LOCAL);
-          Region region = createRegion(name+"_"+whichVM, factory.create());
-          assertNotNull(region);
-          assertNotNull(getRootRegion().getSubregion(name+"_"+whichVM));
-          region.put("KEY-1", "VAL-1");
-          
-          try {
-            testGetConnectedServers_port = startBridgeServer(0);
-          }
-          catch (IOException e) {
-            getLogWriter().error("startBridgeServer threw IOException", e);
-            fail("startBridgeServer threw IOException " + e.getMessage());
-          }
-          
-          assertTrue(testGetConnectedServers_port != 0);
-      
-          getLogWriter().info("[testGetConnectedServers] port=" + 
-            ports[whichVM]);
-          getLogWriter().info("[testGetConnectedServers] serverMemberId=" + 
-            getDistributedMember());
-        }
-      });
-      ports[whichVM] = vm.invokeInt(BridgeMembershipDUnitTest.class, 
-                                    "getTestGetConnectedServers_port");
-      assertTrue(ports[whichVM] != 0);
-    }
-    
-    getLogWriter().info("[testGetConnectedServers] create bridge client");
-    Properties config = new Properties();
-    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    getSystem(config);
-    getCache();
-    
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-
-    for (int i = 0; i < ports.length; i++) {
-      getLogWriter().info("[testGetConnectedServers] creating connectionpool for " + 
-        getServerHostName(host) + " " + ports[i]);
-      int[] thisServerPorts = new int[] { ports[i] };
-      BridgeTestCase.configureConnectionPoolWithName(factory, getServerHostName(host), thisServerPorts, false, -1, -1, null,"pooly"+i);
-      Region region = createRegion(name+"_"+i, factory.create());
-      assertNotNull(getRootRegion().getSubregion(name+"_"+i));
-      region.get("KEY-1");
-    }
-
-    {
-      final int expectedVMCount = host.getVMCount();
-      WaitCriterion wc = new WaitCriterion() {
-        public String description() {
-          return "wait for pools and servers";
-        }
-        public boolean done() {
-          if (PoolManager.getAll().size() != expectedVMCount) {
-            return false;
-          }
-          Map connectedServers = InternalBridgeMembership.getConnectedServers();
-          if (connectedServers == null) {
-            return false;
-          }
-          if (connectedServers.size() != expectedVMCount) {
-            return false;
-          }
-          return true;
-        }
-      };
-      waitForCriterion(wc, 10000, 100, false);
-    }
-
-    {
-      assertEquals(host.getVMCount(), PoolManager.getAll().size());
-      
-    }
-    
-    Map connectedServers = InternalBridgeMembership.getConnectedServers();
-    assertNotNull(connectedServers);
-    assertEquals(host.getVMCount(), connectedServers.size());
-    for (Iterator iter = connectedServers.keySet().iterator(); iter.hasNext();) {
-      String connectedServer = (String) iter.next();
-      getLogWriter().info("[testGetConnectedServers]  value for connectedServer: " + 
-                          connectedServers.get(connectedServer));
-    }
-  }
-
-  protected static int testGetConnectedServers_port;
-  private static int getTestGetConnectedServers_port() {
-    return testGetConnectedServers_port;
-  }
-
-  /**
-   * Tests getConnectedClients(boolean onlyClientsNotifiedByThisServer) where
-   * onlyClientsNotifiedByThisServer is true.
-   */
-  public void testGetNotifiedClients() throws Exception {
-    final Host host = Host.getHost(0);
-    final String name = this.getUniqueName();
-    final int[] ports = new int[host.getVMCount()];
-    
-    for (int i = 0; i < host.getVMCount(); i++) { 
-      final int whichVM = i;
-      final VM vm = Host.getHost(0).getVM(i);
-      vm.invoke(new CacheSerializableRunnable("Create bridge server") {
-        public void run2() throws CacheException {
-          // create BridgeServer in controller vm...
-          getLogWriter().info("[testGetNotifiedClients] Create BridgeServer");
-          getSystem();
-          AttributesFactory factory = new AttributesFactory();
-          Region region = createRegion(name, factory.create());
-          assertNotNull(region);
-          assertNotNull(getRootRegion().getSubregion(name));
-          region.put("KEY-1", "VAL-1");
-          
-          try {
-            testGetNotifiedClients_port = startBridgeServer(0);
-          }
-          catch (IOException e) {
-            getLogWriter().error("startBridgeServer threw IOException", e);
-            fail("startBridgeServer threw IOException " + e.getMessage());
-          }
-          
-          assertTrue(testGetNotifiedClients_port != 0);
-      
-          getLogWriter().info("[testGetNotifiedClients] port=" + 
-            ports[whichVM]);
-          getLogWriter().info("[testGetNotifiedClients] serverMemberId=" + 
-            getMemberId());
-        }
-      });
-      ports[whichVM] = vm.invokeInt(BridgeMembershipDUnitTest.class, 
-                                    "getTestGetNotifiedClients_port");
-      assertTrue(ports[whichVM] != 0);
-    }
-    
-    getLogWriter().info("[testGetNotifiedClients] create bridge client");
-    Properties config = new Properties();
-    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    getSystem(config);
-    getCache();
-    
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-
-    getLogWriter().info("[testGetNotifiedClients] creating connection pool");
-    BridgeTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, -1, null);
-    Region region = createRegion(name, factory.create());
-    assertNotNull(getRootRegion().getSubregion(name));
-    region.registerInterest("KEY-1");
-    region.get("KEY-1");
-
-    final String clientMemberId = getMemberId();
-    
-    pauseForClientToJoin();
-    
-    // assertions go here
-    int[] clientCounts = new int[host.getVMCount()];
-    
-    // only one server vm will have that client for updating
-    for (int i = 0; i < host.getVMCount(); i++) { 
-      final int whichVM = i;
-      final VM vm = Host.getHost(0).getVM(i);
-      vm.invoke(new CacheSerializableRunnable("Create bridge server") {
-        public void run2() throws CacheException {
-          Map clients = InternalBridgeMembership.getConnectedClients(true);
-          assertNotNull(clients);
-          testGetNotifiedClients_clientCount = clients.size();
-          if (testGetNotifiedClients_clientCount > 0) {
-            // assert that the clientMemberId matches
-            assertEquals(clientMemberId, clients.keySet().iterator().next());
-          }
-        }
-      });
-      clientCounts[whichVM] = vm.invokeInt(BridgeMembershipDUnitTest.class, 
-                              "getTestGetNotifiedClients_clientCount");
-    }
-    
-    // only one server should have a notifier for this client...
-    int totalClientCounts = 0;
-    for (int i = 0; i < clientCounts.length; i++) {
-      totalClientCounts += clientCounts[i];
-    }
-    // this assertion fails because the count is 4
-    //assertEquals(1, totalClientCounts);
-  }
-  protected static int testGetNotifiedClients_port;
-  private static int getTestGetNotifiedClients_port() {
-    return testGetNotifiedClients_port;
-  }
-  protected static int testGetNotifiedClients_clientCount;
-  private static int getTestGetNotifiedClients_clientCount() {
-    return testGetNotifiedClients_clientCount;
-  }
-
-  // Simple DistributedMember implementation
-  static final class TestDistributedMember implements DistributedMember {
-    
-    private String host;
-    
-    public TestDistributedMember(String host) {
-      this.host = host;
-    }
-
-    public String getName() {
-      return "";
-    }
-
-    public String getHost() {
-      return this.host;
-    }
-
-    public Set getRoles() {
-      return new HashSet();
-    }
-
-    public int getProcessId() {
-      return 0;
-    }
-
-    public String getId() {
-      return this.host;
-    }
-    
-    public int compareTo(DistributedMember o) {
-      if ((o == null) || !(o instanceof TestDistributedMember)) {
-        throw new InternalGemFireException("Invalidly comparing TestDistributedMember to " + o);
-      }
-      
-      TestDistributedMember tds = (TestDistributedMember) o;
-      return getHost().compareTo(tds.getHost());
-    }
-    
-    public boolean equals(Object obj) {
-      if ((obj == null) || !(obj instanceof TestDistributedMember)) {
-        return false;
-      }
-      return compareTo((TestDistributedMember)obj) == 0;
-    }
-    
-    public int hashCode() {
-      return getHost().hashCode();
-    }
-    
-    public DurableClientAttributes getDurableClientAttributes() {
-      
-      return null;
-    }
-
-    public List<String> getGroups() {
-      return Collections.emptyList();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeMembershipSelectorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeMembershipSelectorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeMembershipSelectorDUnitTest.java
deleted file mode 100644
index 1a7e3ed..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeMembershipSelectorDUnitTest.java
+++ /dev/null
@@ -1,16 +0,0 @@
-package com.gemstone.gemfire.cache30;
-
-/**
- * Same as BridgeMembershipDUnitTest but uses selector in server
- *
- * @author darrel
- * @since 5.1
- */
-public class BridgeMembershipSelectorDUnitTest extends BridgeMembershipDUnitTest {
-  public BridgeMembershipSelectorDUnitTest(String name) {
-    super(name);
-  }
-  protected int getMaxThreads() {
-    return 2;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeTestCase.java
deleted file mode 100644
index 745629e..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/BridgeTestCase.java
+++ /dev/null
@@ -1,376 +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.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.util.BridgeServer;
-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 BridgeServers and bridge clients. This
- * test case was created by refactoring methods from BridgeLoaderTest into
- * this class.
- *
- * @author Kirk Lund
- * @since 4.2.1
- */
-public class BridgeTestCase 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 BridgeTestCase(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();
-    BridgeServer bridge = cache.addBridgeServer();
-    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) {
-    BridgeServer bridge = null;
-    for (Iterator bsI = cache.getBridgeServers().iterator();bsI.hasNext(); ) {
-      bridge = (BridgeServer) 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();
-  }
-
-  /**
-   * Create a bridgeserver 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 BridgeServer
-   * @param rName
-   *          the name of the Region to create on the BridgeServerf
-   * @param port
-   *          the TCP port on which the BridgeServer should listen
-   */
-  public static class BridgeServerCacheLoader extends TestCacheLoader implements Declarable {
-
-    public BridgeServerCacheLoader() {}
-
-    @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";
-  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 BridgeServerCacheLoader());
-        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);
-  }
-
-}
-


[37/38] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-77

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c4b14b72/gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
index 6230937,7bf07c9..2ebe45f
--- 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
@@@ -33,1470 -34,1470 +33,1470 @@@ import com.gemstone.gemfire.i18n.String
   *
   */
  public class LocalizedStrings extends ParentLocalizedStrings {
 -  public static final StringId DataSerializer_NO_INSTANTIATOR_HAS_BEEN_REGISTERED_FOR_CLASS_WITH_ID_0 = new StringIdImpl(3413, "No Instantiator has been registered for class with id  {0}");
 -  public static final StringId DataSerializer_COULD_NOT_INSTANTIATE_AN_INSTANCE_OF_0 = new StringIdImpl(3414, "Could not instantiate an instance of  {0}");
 -  public static final StringId DataSerializer_WHILE_INSTANTIATING_AN_INSTANCE_OF_0 = new StringIdImpl(3415, "While instantiating an instance of  {0}");
 -  public static final StringId Instantiator_CANNOT_REGISTER_A_NULL_CLASS = new StringIdImpl(3416, "Cannot register a null class.");
 -  public static final StringId AdminDistributedSystemImpl_WHILE_WAITING_FOR_FUTURE = new StringIdImpl(3417, "While waiting for Future");
 -  public static final StringId AdminDistributedSystemImpl_THIS_ADMINDISTRIBUTEDSYSTEM_IS_ALREADY_CONNECTED = new StringIdImpl(3418, "This AdminDistributedSystem is already connected.");
 -  public static final StringId AdminDistributedSystemImpl_ONLY_ONE_ADMINDISTRIBUTEDSYSTEM_CONNECTION_CAN_BE_MADE_AT_ONCE = new StringIdImpl(3419, "Only one AdminDistributedSystem connection can be made at once.");
 -  public static final StringId AdminDistributedSystemImpl_AN_ADMINEXCEPTION_WAS_THROWN_WHILE_GETTING_THE_GEMFIRE_HEALTH = new StringIdImpl(3420, "An AdminException was thrown while getting the GemFire health.");
 -  public static final StringId DistributedSystemConfigImpl_ENTITY_CONFIGURATION_XML_FILE_0_DOES_NOT_EXIST = new StringIdImpl(3421, "Entity configuration XML file \"{0}\" does not exist");
 -  public static final StringId ManagedEntityConfigXml_PUBLIC_ID_0_SYSTEM_ID_1 = new StringIdImpl(3422, "Public Id: \"{0}\" System Id: \"{1}\"");
 -  public static final StringId ManagedEntityConfigXmlGenerator_AN_ADMINEXCEPTION_WAS_THROWN_WHILE_GENERATING_XML = new StringIdImpl(3423, "An AdminException was thrown while generating XML.");
 -  public static final StringId ManagedEntityConfigXmlParser_WHILE_PARSING_XML = new StringIdImpl(3424, "While parsing XML");
 -  public static final StringId ManagedEntityConfigXmlParser_MALFORMED_INTEGER_0 = new StringIdImpl(3425, "Malformed integer \"{0}\"");
 -  public static final StringId SystemMemberBridgeServerImpl_UNEXPECTED_EXCEPTION_WHILE_REFRESHING = new StringIdImpl(3426, "Unexpected exception while refreshing");
 -  public static final StringId AgentImpl_WHILE_CREATING_OBJECTNAME_0 = new StringIdImpl(3427, "While creating ObjectName:  {0}");
 -  public static final StringId AgentImpl_COULD_NOT_OPEN_LOG_FILE_0 = new StringIdImpl(3428, "Could not open log file \"{0}\"");
 -  public static final StringId GemFireHealthJmxImpl_WHILE_GETTING_THE_DISTRIBUTEDSYSTEMHEALTHCONFIG = new StringIdImpl(3429, "While getting the DistributedSystemHealthConfig");
 -  public static final StringId GemFireHealthJmxImpl_WHILE_GETTING_THE_GEMFIREHEALTHCONFIG = new StringIdImpl(3430, "While getting the GemFireHealthConfig");
 -  public static final StringId AbstractDistributionConfig_INVALID_LOCATOR_0_THE_PORT_1_WAS_NOT_GREATER_THAN_ZERO_AND_LESS_THAN_65536 = new StringIdImpl(3431, "Invalid locator \"{0}\". The port {1} was not greater than zero and less than 65,536.");
 -  public static final StringId DistributionManager_INTERRUPTED_WHILE_WAITING_FOR_FIRST_STARTUPRESPONSEMESSAGE = new StringIdImpl(3432, "Interrupted while waiting for first StartupResponseMessage");
 -  public static final StringId DistributionManager_RECEIVED_NO_CONNECTION_ACKNOWLEDGMENTS_FROM_ANY_OF_THE_0_SENIOR_CACHE_MEMBERS_1 = new StringIdImpl(3433, "Received no connection acknowledgments from any of the  {0}  senior cache members:  {1}");
 -  public static final StringId MessageFactory_AN_INSTANTIATIONEXCEPTION_WAS_THROWN_WHILE_INSTANTIATING_A_0 = new StringIdImpl(3434, "An InstantiationException was thrown while instantiating a  {0}");
 -  public static final StringId MessageFactory_COULD_NOT_ACCESS_ZEROARG_CONSTRUCTOR_OF_0 = new StringIdImpl(3435, "Could not access zero-arg constructor of  {0}");
 -  public static final StringId JGroupMembershipManager_AN_IOEXCEPTION_WAS_THROWN_WHILE_OPENING_0 = new StringIdImpl(3436, "An IOException was thrown while opening \"{0}\"");
 -  public static final StringId JGroupMembershipManager_AN_EXCEPTION_WAS_THROWN_WHILE_READING_JGROUPS_CONFIG = new StringIdImpl(3437, "An Exception was thrown while reading JGroups config.");
 -  public static final StringId JGroupMembershipManager_AN_EXCEPTION_WAS_THROWN_WHILE_CONNECTING_TO_JGROUPS = new StringIdImpl(3438, "An Exception was thrown while connecting to Jgroups.");
 -  public static final StringId InternalInstantiator_CANNOT_UNREGISTER_A_NULL_CLASS = new StringIdImpl(3439, "Cannot unregister a null class");
 -  public static final StringId StatisticDescriptorImpl_THE_STATISTIC_0_WITH_ID_1_IS_OF_TYPE_2_AND_IT_WAS_EXPECTED_TO_BE_AN_INT = new StringIdImpl(3440, "The statistic  {0}  with id  {1}  is of type  {2}  and it was expected to be an int.");
 -  public static final StringId StatisticDescriptorImpl_THE_STATISTIC_0_WITH_ID_1_IS_OF_TYPE_2_AND_IT_WAS_EXPECTED_TO_BE_A_DOUBLE = new StringIdImpl(3441, "The statistic  {0}  with id  {1}  is of type  {2}  and it was expected to be a double.");
 -  public static final StringId StatisticsTypeXml_SAX_ERROR_WHILE_WORKING_WITH_XML = new StringIdImpl(3442, "SAX error while working with XML");
 -  public static final StringId StatisticsTypeXml_SAX_FATAL_ERROR_WHILE_WORKING_WITH_XML = new StringIdImpl(3443, "SAX fatal error while working with XML");
 -  public static final StringId AdminRequest_A_REPLYEXCEPTION_WAS_THROWN_WHILE_WAITING_FOR_A_REPLY = new StringIdImpl(3444, "A ReplyException was thrown while waiting for a reply.");
 -  public static final StringId AdminWaiters_ADMINISTRATION_REQUEST_0_SENT_TO_1_TIMED_OUT_AFTER_2_SECONDS = new StringIdImpl(3445, "Administration request  {0}  sent to  {1}  timed out after  {2}  seconds.");
 -  public static final StringId RemoteAlert_INVALIDATE_TIMESTAMP_0 = new StringIdImpl(3446, "Invalidate timestamp:  {0}");
 -  public static final StringId RemoteGemFireVM_AN_EXCEPTION_WAS_THROWN_WHILE_CREATING_VM_ROOT_REGION_0 = new StringIdImpl(3447, "An Exception was thrown while creating VM root region \"{0}\"");
 -  public static final StringId RemoteGemFireVM_WHILE_CREATING_SUBREGION_0_OF_1 = new StringIdImpl(3448, "While creating subregion \"{0}\" of \"{1}\"");
 -  public static final StringId RemoteGfManagerAgent_AN_EXCEPUTIONEXCEPTION_WAS_THROWN_WHILE_WAITING_FOR_FUTURE = new StringIdImpl(3449, "An ExceputionException was thrown while waiting for Future.");
 -  public static final StringId AbstractLRURegionMap_INCREMENT_TO_LRU_DELTA_WITHOUT_USE_OF_PRIOR_VALUE_PUT_AND_REPLACE_CALLS_MUST_BE_FOLLOWED_BY_A_CALL_TO_LRUUPDATECALLBACK_DELTA_IS_0 = new StringIdImpl(3450, "Increment to lru delta without use of prior value, put and replace calls must be followed by a call to lruUpdateCallback. (delta is  {0} ).");
 -  public static final StringId BucketAdvisor_CANNOT_CHANGE_FROM_0_TO_1_FOR_BUCKET_2 = new StringIdImpl(3451, "Cannot change from  {0}  to  {1}  for bucket  {2}");
 -  public static final StringId GatewayEventRemoteDispatcher_0_EXCEPTION_DURING_PROCESSING_BATCH_1_ON_CONNECTION_2 = new StringIdImpl(3452, "{0} : Exception during processing batch  {1}  on connection  {2}");
 -  public static final StringId GemFireCache_COULD_NOT_CONVERT_XML_FILE_0_TO_AN_URL = new StringIdImpl(3453, "Could not convert XML file \"{0}\" to an URL.");
 -  public static final StringId GemFireCache_WHILE_OPENING_CACHE_XML_0_THE_FOLLOWING_ERROR_OCCURRED_1 = new StringIdImpl(3454, "While opening Cache XML \"{0}\" the following error occurred \"{1}\"");
 -  public static final StringId Oplog_TRIED_TO_SEEK_TO_0_BUT_THE_FILE_LENGTH_IS_1_OPLOG_FILE_OBJECT_USED_FOR_READING_2 = new StringIdImpl(3455, "Tried to seek to {0}, but the file length is {1}. Oplog File object used for reading={2}");
 -  public static final StringId PartitionedRegion_ATTEMPT_TO_ACQUIRE_PRIMARY_NODE_FOR_WRITE_ON_BUCKET_0_TIMED_OUT_IN_1_MS_CURRENT_REDUNDANCY_2_DOES_NOT_SATISFY_MINIMUM_3 = new StringIdImpl(3456, "Attempt to acquire primary node for write on bucket  {0}  timed out in  {1}  ms. Current redundancy [ {2} ] does not satisfy minimum [ {3} ]");
 -  public static final StringId PartitionedRegion_ATTEMPT_TO_ACQUIRE_PRIMARY_NODE_FOR_READ_ON_BUCKET_0_TIMED_OUT_IN_1_MS_CURRENT_REDUNDANCY_2_DOES_NOT_SATISFY_MINIMUM_3 = new StringIdImpl(3457, "Attempt to acquire primary node for read on bucket  {0}  timed out in  {1}  ms. Current redundancy [ {2} ] does not satisfy minimum [ {3} ]");
 -  public static final StringId PartitionedRegion_ATTEMPT_TO_ACQUIRE_PRIMARY_NODE_FOR_READ_ON_BUCKET_0_TIMED_OUT_IN_1_MS = new StringIdImpl(3458, "Attempt to acquire primary node for read on bucket  {0}  timed out in  {1}  ms");
 -
 -  public static final StringId RegionAttributesCreation_CONCURRENCYCHECKSENABLED_IS_NOT_THE_SAME = new StringIdImpl(3459, "ConcurrencyChecksEnabled is not the same");
 -  public static final StringId RegionAttributesCreation_ENABLE_OFF_HEAP_MEMORY_IS_NOT_THE_SAME = new StringIdImpl(3460, "EnableOffHeapMemory is not the same");
 -
 -  public static final StringId ProxyRegionMap_NO_ENTRY_SUPPORT_ON_REGIONS_WITH_DATAPOLICY_0 = new StringIdImpl(3461, "No entry support on regions with DataPolicy  {0}");
 -  public static final StringId SearchLoadAndWriteProcessor_WHILE_INVOKING_A_REMOTE_NETLOAD_0 = new StringIdImpl(3462, "While invoking a remote netLoad:  {0}");
 -  public static final StringId SearchLoadAndWriteProcessor_WHILE_INVOKING_A_REMOTE_NETWRITE_0 = new StringIdImpl(3463, "While invoking a remote netWrite:  {0}");
 -  public static final StringId SearchLoadAndWriteProcessor_TIMED_OUT_WHILE_DOING_NETSEARCHNETLOADNETWRITE_PROCESSORID_0_KEY_IS_1 = new StringIdImpl(3464, "Timed out while doing netsearch/netload/netwrite processorId= {0}  Key is  {1}");
 -  public static final StringId SearchLoadAndWriteProcessor_TIMEOUT_DURING_NETSEARCHNETLOADNETWRITE_DETAILS_0 = new StringIdImpl(3465, "Timeout during netsearch/netload/netwrite. Details:  {0}");
 -  public static final StringId TXCommitMessage_THESE_REGIONS_EXPERIENCED_RELIABILITY_FAILURE_DURING_DISTRIBUTION_OF_THE_OPERATION_0 = new StringIdImpl(3466, "These regions experienced reliability failure during distribution of the operation:  {0}");
 -  public static final StringId TXCommitMessage_COMMIT_OPERATION_GENERATED_ONE_OR_MORE_EXCEPTIONS_FROM_0 = new StringIdImpl(3467, "Commit operation generated one or more exceptions from  {0}");
 -  public static final StringId BucketSizeMessage_BUCKETSIZERESPONSE_GOT_REMOTE_CACHECLOSEDEXCEPTION_FORCING_REATTEMPT = new StringIdImpl(3468, "BucketSizeResponse got remote CacheClosedException; forcing reattempt.");
 -  public static final StringId BucketSizeMessage_BUCKETSIZERESPONSE_GOT_REMOTE_REGION_DESTROYED_FORCING_REATTEMPT = new StringIdImpl(3469, "BucketSizeResponse got remote Region destroyed; forcing reattempt.");
 -  public static final StringId ContainsKeyValueMessage_CONTAINSKEYVALUERESPONSE_GOT_REMOTE_CACHEEXCEPTION_FORCING_REATTEMPT = new StringIdImpl(3470, "ContainsKeyValueResponse got remote CacheException; forcing reattempt.");
 -  public static final StringId DumpB2NRegion_B2NRESPONSE_GOT_REMOTE_CACHEEXCEPTION_THROWING_FORCEREATTEMPTEXCEPTION = new StringIdImpl(3471, "B2NResponse got remote CacheException, throwing ForceReattemptException.");
 -  public static final StringId FetchEntriesMessage_FETCHKEYSRESPONSE_GOT_REMOTE_CANCELLATION_FORCING_REATTEMPT = new StringIdImpl(3472, "FetchKeysResponse got remote cancellation; forcing reattempt.");
 -  public static final StringId FetchEntryMessage_FETCHENTRYRESPONSE_GOT_REMOTE_CACHEEXCEPTION_FORCING_REATTEMPT = new StringIdImpl(3473, "FetchEntryResponse got remote CacheException; forcing reattempt.");
 -  public static final StringId FetchKeysMessage_FETCHKEYSRESPONSE_GOT_REMOTE_CACHECLOSEDEXCEPTION_FORCING_REATTEMPT = new StringIdImpl(3474, "FetchKeysResponse got remote CacheClosedException; forcing reattempt.");
 -  public static final StringId GetMessage_UNABLE_TO_DESERIALIZE_VALUE_IOEXCEPTION = new StringIdImpl(3475, "Unable to deserialize value (IOException)");
 -  public static final StringId GetMessage_UNABLE_TO_DESERIALIZE_VALUE_CLASSNOTFOUNDEXCEPTION = new StringIdImpl(3476, "Unable to deserialize value (ClassNotFoundException)");
 -  public static final StringId RemoteOperationMessage_0_COULD_NOT_FIND_REGION_1 = new StringIdImpl(3477, "{0} : could not find region {1}");
 -  public static final StringId ManageBucketMessage_NODERESPONSE_GOT_REMOTE_CANCELLATION_THROWING_PARTITIONEDREGIONCOMMUNICATION_EXCEPTION = new StringIdImpl(3478, "NodeResponse got remote cancellation, throwing PartitionedRegionCommunication Exception.");
 -  public static final StringId ManageBucketMessage_NODERESPONSE_GOT_LOCAL_DESTROY_ON_THE_PARTITIONREGION_THROWING_FORCEREATTEMPTEXCEPTION = new StringIdImpl(3479, "NodeResponse got local destroy on the PartitionRegion , throwing ForceReattemptException.");
 -  public static final StringId PartitionMessage_PARTITIONRESPONSE_GOT_REMOTE_CACHECLOSEDEXCEPTION = new StringIdImpl(3480, "PartitionResponse got remote CacheClosedException");
 -  public static final StringId PrimaryRequestMessage_NODERESPONSE_GOT_REMOTE_CACHECLOSEDEXCEPTION_THROWING_PARTITIONEDREGIONCOMMUNICATION_EXCEPTION = new StringIdImpl(3481, "NodeResponse got remote CacheClosedException, throwing PartitionedRegionCommunication Exception.");
 -  public static final StringId ConnectionImpl_INVALID_PING_REPLY_WITH_EXPECTED_TRANSACTION_0_AND_ACTUAL_TRANSACTION_1 = new StringIdImpl(3482, "Invalid ping reply with expected transaction  {0}  and actual transaction  {1}");
 -  public static final StringId ConnectionImpl_EXCEPTION_DURING_PING_WITH_TRANSACTION_0_DUE_TO_THE_FOLLOWING_SERVER_EXCEPTION = new StringIdImpl(3483, "Exception during ping with transaction  {0}  due to the following server exception:");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_GET = new StringIdImpl(3484, "{0} : While performing a remote get.");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_PUT = new StringIdImpl(3485, "{0} : While performing a remote put.");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_DESTROY = new StringIdImpl(3486, "{0} : While performing a remote destroy.");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_DESTROY_REGION = new StringIdImpl(3487, "{0} : While performing a remote destroy region.");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_CLEAR_REGION = new StringIdImpl(3488, "{0} : While performing a remote clear region");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_CREATE_REGION = new StringIdImpl(3489, "{0} : While performing a remote create region");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_UNREGISTERINTEREST = new StringIdImpl(3490, "{0} : While performing a remote unregisterInterest");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_CONTAINSKEY = new StringIdImpl(3491, "{0} : While performing a remote containsKey");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_PROCESSBATCH = new StringIdImpl(3492, "{0} : While performing a remote processBatch");
 -  public static final StringId ConnectionImpl_EXCEPTION_OCCURED_ON_SERVER_ERROR_0 = new StringIdImpl(3493, "Exception occurred on Server. Error : {0}.");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_REGISTERINTEREST = new StringIdImpl(3494, "{0} : While performing a remote registerInterest");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_REGISTERINTEREST_1 = new StringIdImpl(3495, "{0} : While performing a remote registerInterest:  {1}");
 -  public static final StringId LicenseInfo_EVAL_NAG = new StringIdImpl(3496, "NOTICE: use of GemFire with this license key is only permitted in a non-production environment and for the period, if any, limited by the license key.  Notwithstanding any other provision in the EULA, this Evaluation License of GemFire is provided AS-IS without support or warranty of any kind, expressed or implied.");
 -  public static final StringId ConnectionImpl_0_WHILE_DESERIALIZING_QUERY_RESULT = new StringIdImpl(3497, "{0} : While deserializing query result");
 -  public static final StringId TombstoneService_UNEXPECTED_EXCEPTION = new StringIdImpl(3498, "GemFire garbage collection service encountered an unexpected exception");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_KEYSET_1 = new StringIdImpl(3499, "{0} : While performing a remote keySet {1}");
 -  public static final StringId ConnectionImpl_0_WHILE_PERFORMING_A_REMOTE_KEYSET = new StringIdImpl(3500, "{0} : While performing a remote keySet");
 -  public static final StringId ConnectionImpl_0_WHILE_MAKE_PRIMARY = new StringIdImpl(3501, "{0} : While make primary");
 -  public static final StringId ConnectionImpl_CONNECTION_0_WHILE_SENDING_PERIODIC_ACK = new StringIdImpl(3502, "Connection ( {0} ): While sending periodic ack");
 -  public static final StringId Oplog_FAILED_READING_FROM_0_OPLOG_DETAILS_1_2_3_4_5_6 = new StringIdImpl(3503, "Failed reading from \"{0}\". \n oplogID = {1}\n Offset being read={2} Current Oplog Size={3}  Actual File Size ={4} IS ASYNCH MODE ={5} IS ASYNCH WRITER ALIVE={6}");
 -  public static final StringId EndpointImpl_0_CAN_NOT_REGISTER_INTEREST_IN_REGION_1_KEY_2_BECAUSE_THE_SERVER_IS_NOT_AVAILABLE = new StringIdImpl(3504, "{0} : Can not register interest in region  {1}  key  {2}  because the server is not available.");
 -  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_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");
 -  public static final StringId CacheXmlParser_WHILE_PARSING_XML = new StringIdImpl(3512, "While parsing XML");
 -  public static final StringId CacheXmlParser_MALFORMED_INTEGER_0 = new StringIdImpl(3513, "Malformed integer \"{0}\"");
 -  public static final StringId CacheXmlParser_MALFORMED_FLOAT_0 = new StringIdImpl(3514, "Malformed float \"{0}\"");
 -  public static final StringId CacheXmlParser_COULD_NOT_CREATE_GATEWAY_WITH_ID_0 = new StringIdImpl(3515, "Could not create gateway with id  {0}");
 -  public static final StringId CacheXmlParser_COULD_NOT_ADD_ENDPOINT_WITH_ID_0 = new StringIdImpl(3516, "Could not add endpoint with id  {0}");
 -  public static final StringId CacheXmlParser_COULD_NOT_LOAD_KEYCONSTRAINT_CLASS_0 = new StringIdImpl(3517, "Could not load key-constraint class:  {0}");
 -  public static final StringId CacheXmlParser_COULD_NOT_LOAD_VALUECONSTRAINT_CLASS_0 = new StringIdImpl(3518, "Could not load value-constraint class:  {0}");
 -  public static final StringId CacheXmlParser_WHILE_INSTANTIATING_A_0 = new StringIdImpl(3519, "While instantiating a  {0}");
 -  public static final StringId RegionCreation_GETTING_ENTRIES_RECURSIVELY_IS_NOT_SUPPORTED = new StringIdImpl(3520, "Getting entries recursively is not supported.");
 -  public static final StringId AbstractPoolCache_ABSTRACTPOOLEDCACHEGETPOOLEDCONNECTIONFROMPOOLINTERRUPTEDEXCEPTION_IN_WAITING_THREAD = new StringIdImpl(3521, "AbstractPooledCache::getPooledConnectionFromPool:InterruptedException in waiting thread");
 -  public static final StringId ConnectionPoolCacheImpl_CONNECTIONPOOLCACHEIMPLGENEWCONNECTION_EXCEPTION_IN_CREATING_NEW_POOLEDCONNECTION = new StringIdImpl(3522, "ConnectionPoolCacheImpl::getNewConnection: Exception in creating new PooledConnection");
 -  public static final StringId ConnectionPoolCacheImpl_CONNECTIONPOOLCACHEIMPLGENEWCONNECTION_CONNECTIONPOOLCACHE_NOT_INTIALIZED_WITH_CONNECTIONPOOLDATASOURCE = new StringIdImpl(3523, "ConnectionPoolCacheImpl::getNewConnection: ConnectionPoolCache not initialized with ConnectionPoolDatasource");
 -  public static final StringId FacetsJCAConnectionManagerImpl_FACETSJCACONNECTIONMANAGERIMPL_ALLOCATECONNECTION_IN_GETTING_CONNECTION_FROM_POOL_DUE_TO_0 = new StringIdImpl(3524, "FacetsJCAConnectionManagerImpl:: allocateConnection : in getting connection from pool due to  {0}");
 -  public static final StringId FacetsJCAConnectionManagerImpl_FACETSJCACONNECTIONMANAGERIMPL_ALLOCATECONNECTION_SYSTEM_EXCEPTION_DUE_TO_0 = new StringIdImpl(3525, "FacetsJCAConnectionManagerImpl:: allocateConnection :system exception due to  {0}");
 -  //public static final StringId GemFireBasicDataSource_GEMFIREBASICDATASOURCEGETCONNECTIONEXCEPTION_WHILE_TRYING_TO_LOAD_THE_DRIVER_DUE_TO_0 = new StringIdImpl(3526, "GemFireBasicDataSource::getConnection:Exception while trying to load the driver due to  {0}");
 -  public static final StringId JCAConnectionManagerImpl_JCACONNECTIONMANAGERIMPL_ALLOCATECONNECTION_IN_GETTING_CONNECTION_FROM_POOL_DUE_TO_0 = new StringIdImpl(3527, "JCAConnectionManagerImpl:: allocateConnection : in getting connection from pool due to  {0}");
 -  public static final StringId JCAConnectionManagerImpl_JCACONNECTIONMANAGERIMPL_ALLOCATECONNECTION_IN_TRANSACTION_DUE_TO_0 = new StringIdImpl(3528, "JCAConnectionManagerImpl:: allocateConnection : in transaction due to  {0}");
 -  public static final StringId JCAConnectionManagerImpl_JCACONNECTIONMANAGERIMPL_ALLOCATECONNECTION_SYSTEM_EXCEPTION_DUE_TO_0 = new StringIdImpl(3529, "JCAConnectionManagerImpl:: allocateConnection :system exception due to  {0}");
 -  public static final StringId ManagedPoolCacheImpl_MANAGEDPOOLCACHEIMPLGETNEWCONNECTION_EXCEPTION_IN_CREATING_NEW_MANAGED_POOLEDCONNECTION = new StringIdImpl(3530, "ManagedPoolCacheImpl::getNewConnection: Exception in creating new Managed PooledConnection");
 -  public static final StringId TranxPoolCacheImpl_TRANXPOOLCACHEIMPLGETNEWCONNECTION_EXCEPTION_IN_CREATING_NEW_TRANSACTION_POOLEDCONNECTION = new StringIdImpl(3531, "TranxPoolCacheImpl::getNewConnection: Exception in creating new transaction PooledConnection");
 -  public static final StringId TranxPoolCacheImpl_TRANXPOOLCACHEIMPLGETNEWCONNECTION_CONNECTIONPOOLCACHE_NOT_INTIALIZED_WITH_XADATASOURCE = new StringIdImpl(3532, "TranxPoolCacheImpl::getNewConnection: ConnectionPoolCache not intialized with XADatasource");
 -  public static final StringId ObjectXmlExporter_A_TRANFSFORMEREXCEPTION_WAS_THROWN_WHILE_GENERATING_XML_FOR_A_0 = new StringIdImpl(3533, "A TranfsformerException was thrown while generating XML for a  {0}");
 -  public static final StringId ObjectXmlExporter_AN_EXCEPTION_WAS_THROWN_WHILE_ACCESSING_FIELD_0_IN_CLASS_1 = new StringIdImpl(3534, "An Exception was thrown while accessing field  {0}  in class  {1}");
 -  public static final StringId ObjectXmlImporter_AN_IOEXCEPTION_WAS_THROWN_WHILE_PARSING_XML = new StringIdImpl(3535, "An IOException was thrown while parsing XML.");
 -  public static final StringId ObjectXmlImporter_A_PARSERCONFIGURATIONEXCEPTION_WAS_THROWN_WHILE_PARSING_XML = new StringIdImpl(3536, "A ParserConfigurationException was thrown While parsing XML.");
 -  public static final StringId ObjectXmlImporter_A_SAXEXCEPTION_WAS_THROWN_WHILE_PARSING_XML = new StringIdImpl(3537, "A SAXException was thrown While parsing XML.");
 -  public static final StringId ObjectXmlImporter_COULD_NOT_LOAD_CLASS_0 = new StringIdImpl(3538, "Could not load class  {0}");
 -  public static final StringId ObjectXmlImporter_AN_EXCEPTION_WAS_THROWN_WHILE_INSTANTIATING_0 = new StringIdImpl(3539, "An Exception was thrown while instantiating  {0}");
 -  public static final StringId ObjectXmlImporter_AN_EXCEPTION_WAS_THROWN_WHILE_LOCATING_FIELD_0_IN_CLASS_1 = new StringIdImpl(3540, "An Exception was thrown while locating field  {0}  in class  {1}");
 -  public static final StringId ObjectXmlImporter_A_CLASSNOTFOUNDEXCEPTION_AS_THROWN_WHILE_LOADING_ARRAY_ELEMENT_TYPE_0 = new StringIdImpl(3541, "A ClassNotFoundException was thrown while loading array element type:  {0}");
 -
 -  public static final StringId DataSerializer_UNKNOWN_PRIMITIVE_TYPE_0 = new StringIdImpl(3542, "unknown primitive type:  {0}");
 -  public static final StringId DataSerializer_UNEXPECTED_TYPECODE_0 = new StringIdImpl(3543, "unexpected typeCode:  {0}");
 -  public static final StringId DataSerializer_CONFLICTING_NONPUBLIC_INTERFACE_CLASS_LOADERS = new StringIdImpl(3544, "Conflicting non-public interface class loaders");
 -  public static final StringId SystemFailure_0_MEMORY_HAS_REMAINED_CHRONICALLY_BELOW_1_BYTES_OUT_OF_A_MAXIMUM_OF_2_FOR_3_SEC = new StringIdImpl(3545, "{0} : memory has remained chronically below  {1}  bytes (out of a maximum of  {2} ) for  {3}  sec.");
 -  public static final StringId SystemFailure_JVM_CORRUPTION_HAS_BEEN_DETECTED = new StringIdImpl(3546, "JVM corruption has been detected");
 -  public static final StringId LonerDistributionManager_CANNOT_RESOLVE_LOCAL_HOST_NAME_TO_AN_IP_ADDRESS = new StringIdImpl(3547, "Cannot resolve local host name to an IP address");
 -  public static final StringId AbstractDistributionConfig_UNEXPECTED_PROBLEM_GETTING_INETADDRESS_0 = new StringIdImpl(3548, "Unexpected problem getting inetAddress: {0}");
 -  public static final StringId DistributionManager_UNKNOWN_MEMBER_TYPE_0 = new StringIdImpl(3549, "Unknown  member type:  {0}");
 -  public static final StringId DistributionManager_UNKNOWN_PROCESSOR_TYPE = new StringIdImpl(3550, "unknown processor type {0}");
 -  public static final StringId DLockRequestProcessor_UNKNOWN_RESPONSE_CODE_0 = new StringIdImpl(3551, "Unknown response code {0}");
 -  public static final StringId StreamingOperation_THIS_SHOULDNT_HAPPEN = new StringIdImpl(3552, "this should not happen");
 -  public static final StringId MemberAttributes_IOEXCEPTION_ON_A_BYTE_ARRAY_0 = new StringIdImpl(3553, "IOException on a byte array:  {0}");
 -  public static final StringId MemberAttributes_CLASSNOTFOUNDEXCEPTION_IN_DESERIALIZATION_0 = new StringIdImpl(3554, "ClassNotFoundException in deserialization:  {0}");
 -  public static final StringId JGroupMembershipManager_UNKNOWN_STARTUP_EVENT_0 = new StringIdImpl(3555, "unknown startup event:  {0}");
 -  public static final StringId JGroupMember_ATTEMPT_TO_EXTERNALIZE_NULL_IP_ADDRESS = new StringIdImpl(3556, "attempt to externalize null ip address");
 -  public static final StringId UpdateOperation_UNKNOWN_DESERIALIZATION_POLICY = new StringIdImpl(3557, "unknown deserialization policy");
 -  public static final StringId UpdateOperation_UNKNOWN_DESERIALIZATION_POLICY_0 = new StringIdImpl(3558, "unknown deserialization policy:  {0}");
 -  public static final StringId QueuedOperation_CACHEWRITER_SHOULD_NOT_BE_CALLED = new StringIdImpl(3559, "CacheWriter should not be called");
 -  public static final StringId QueuedOperation_DISTRIBUTEDLOCK_SHOULD_NOT_BE_ACQUIRED = new StringIdImpl(3560, "DistributedLock should not be acquired");
 -  public static final StringId InitialImageOperation_ALREADY_PROCESSED_LAST_CHUNK = new StringIdImpl(3561, "Already processed last chunk");
 -  public static final StringId PartitionedRegionQueryEvaluator_UNABLE_TO_GET_NODE_FOR_BUCKET_ID_0 = new StringIdImpl(3562, "unable to get node for bucket id  {0}");
 -  public static final StringId DiskEntry_WRITE_TO_DISK_SHOULD_NOT_HAVE_BEEN_INVOKED_IN_THE_ENTRY_INITIALIZATION_PHASE = new StringIdImpl(3563, "Write to disk should not have been invoked in the entry initialization phase");
 -  public static final StringId DistributedCacheOperation_SHOULD_NEVER_BE_CALLED = new StringIdImpl(3564, "should never be called");
 -  public static final StringId ExpiryTask_UNRECOGNIZED_EXPIRATION_ACTION_0 = new StringIdImpl(3565, "unrecognized expiration action:  {0}");
 -  public static final StringId LocalRegion_UNEXPECTED_EXCEPTION = new StringIdImpl(3566, "unexpected exception");
 -  public static final StringId LocalRegion_CACHE_WRITER_SHOULD_NOT_HAVE_BEEN_CALLED_FOR_LOCALDESTROY = new StringIdImpl(3567, "Cache Writer should not have been called for localDestroy");
 -  public static final StringId LocalRegion_NO_DISTRIBUTED_LOCK_SHOULD_HAVE_BEEN_ATTEMPTED_FOR_LOCALDESTROY = new StringIdImpl(3568, "No distributed lock should have been attempted for localDestroy");
 -  public static final StringId LocalRegion_CACHEWRITEREXCEPTION_SHOULD_NOT_BE_THROWN_IN_LOCALDESTROYREGION = new StringIdImpl(3569, "CacheWriterException should not be thrown in localDestroyRegion");
 -  public static final StringId LocalRegion_TIMEOUTEXCEPTION_SHOULD_NOT_BE_THROWN_IN_LOCALDESTROYREGION = new StringIdImpl(3570, "TimeoutException should not be thrown in localDestroyRegion");
 -  public static final StringId LocalRegion_UNKNOWN_INTEREST_TYPE = new StringIdImpl(3571, "unknown interest type");
 -  public static final StringId LocalRegion_NOT_YET_SUPPORTED = new StringIdImpl(3572, "not yet supported");
 -  public static final StringId LocalRegion_GOT_REGIONEXISTSEXCEPTION_IN_REINITIALIZE_WHEN_HOLDING_DESTROY_LOCK = new StringIdImpl(3573, "Got RegionExistsException in reinitialize when holding destroy lock");
 -  public static final StringId LocalRegion_UNEXPECTED_THREADINITLEVELREQUIREMENT = new StringIdImpl(3574, "Unexpected threadInitLevelRequirement");
 -  public static final StringId LocalRegion_ENTRY_ALREADY_EXISTED_0 = new StringIdImpl(3575, "Entry already existed:  {0}");
 -  public static final StringId LocalRegion_CACHE_WRITER_SHOULD_NOT_HAVE_BEEN_CALLED_FOR_EVICTDESTROY = new StringIdImpl(3576, "Cache Writer should not have been called for evictDestroy");
 -  public static final StringId LocalRegion_NO_DISTRIBUTED_LOCK_SHOULD_HAVE_BEEN_ATTEMPTED_FOR_EVICTDESTROY = new StringIdImpl(3577, "No distributed lock should have been attempted for evictDestroy");
 -  public static final StringId LocalRegion_ENTRYNOTFOUNDEXCEPTION_SHOULD_BE_MASKED_FOR_EVICTDESTROY = new StringIdImpl(3578, "EntryNotFoundException should be masked for evictDestroy");
 -  public static final StringId LocalRegion_CACHEWRITEREXCEPTION_SHOULD_NOT_BE_THROWN_HERE = new StringIdImpl(3579, "CacheWriterException should not be thrown here");
 -  public static final StringId LocalRegion_TIMEOUTEXCEPTION_SHOULD_NOT_BE_THROWN_HERE = new StringIdImpl(3580, "TimeoutException should not be thrown here");
 -  public static final StringId GemFireCache_CACHE_SHOULD_NOT_BE_CLOSED_YET = new StringIdImpl(3581, "cache should not be closed yet");
 -  public static final StringId GemFireCache_UNEXPECTED_EXCEPTION = new StringIdImpl(3582, "unexpected exception");
 -  public static final StringId EntryEvents_MUST_NOT_SERIALIZE_0_IN_THIS_CONTEXT = new StringIdImpl(3583, "Must not serialize {0} in this context.");
 -  public static final StringId DistributedRegion_IF_LOADING_A_SNAPSHOT_THEN_SHOULD_NOT_BE_RECOVERING_ISRECOVERING_0_SNAPSHOTSTREAM_1 = new StringIdImpl(3584, "if loading a snapshot, then should not be recovering; isRecovering= {0} ,snapshotStream= {1}");
 -  public static final StringId AbstractUpdateOperation_CACHEWRITER_SHOULD_NOT_BE_CALLED = new StringIdImpl(3585, "CacheWriter should not be called");
 -  public static final StringId AbstractUpdateOperation_DISTRIBUTEDLOCK_SHOULD_NOT_BE_ACQUIRED = new StringIdImpl(3586, "DistributedLock should not be acquired");
 -  public static final StringId RegionEventImpl_CLONE_IS_SUPPORTED = new StringIdImpl(3587, "clone IS supported");
 -  public static final StringId DestroyOperation_CACHEWRITER_SHOULD_NOT_BE_CALLED = new StringIdImpl(3588, "CacheWriter should not be called");
 -  public static final StringId DestroyOperation_DISTRIBUTEDLOCK_SHOULD_NOT_BE_ACQUIRED = new StringIdImpl(3589, "DistributedLock should not be acquired");
 -  public static final StringId StreamingPartitionOperation_UNEXPECTED_CONDITION = new StringIdImpl(3590, "unexpected condition");
 -  public static final StringId PartitionMessage_SORRY_USE_OPERATEONPARTITIONEDREGION_FOR_PR_MESSAGES = new StringIdImpl(3591, "Sorry, use operateOnPartitionedRegion for PR messages");
 -  public static final StringId FetchEntryMessage_FETCHENTRYMESSAGE_MESSAGE_SENT_TO_WRONG_MEMBER = new StringIdImpl(3592, "FetchEntryMessage message sent to wrong member");
 -  public static final StringId GetMessage_GET_MESSAGE_SENT_TO_WRONG_MEMBER = new StringIdImpl(3593, "Get message sent to wrong member");
 -  public static final StringId FetchEntriesMessage_ALREADY_PROCESSED_LAST_CHUNK = new StringIdImpl(3594, "Already processed last chunk");
 -  public static final StringId FetchKeysMessage_ALREADY_PROCESSED_LAST_CHUNK = new StringIdImpl(3595, "Already processed last chunk");
 -  public static final StringId CacheClientProxy_UNKNOWN_INTEREST_TYPE = new StringIdImpl(3596, "Unknown interest type");
 -  public static final StringId CacheClientProxy_BAD_INTEREST_TYPE = new StringIdImpl(3597, "bad interest type");
 -  public static final StringId InternalDistributedSystem_PROBLEM_IN_INITIALIZING_KEYS_FOR_CLIENT_AUTHENTICATION = new StringIdImpl(3598, "Problem in initializing keys for client authentication");
 -  public static final StringId ConnectionImpl_UNKNOWN_MESSAGE_TYPE_0 = new StringIdImpl(3599, "Unknown message type {0}");
 -  public static final StringId ConnectionProxyImpl_UNKNOWN_INTERESTTYPE = new StringIdImpl(3600, "Unknown interestType");
 -  public static final StringId ConnectionProxyImpl_WEIRD_ENDPOINT_TYPE_0 = new StringIdImpl(3601, "Weird endpoint type  {0}");
 -  public static final StringId BaseCommand_NOT_YET_SUPPORTED = new StringIdImpl(3602, "not yet supported");
 -  public static final StringId BaseCommand_UNKNOWN_INTEREST_TYPE = new StringIdImpl(3603, "unknown interest type");
 -  public static final StringId SystemAdmin_REGION_OPTION_HELP = new StringIdImpl(3604, "Used to specify what region an operation is to be done on.");
 -  public static final StringId EntryNotFoundInRegion_THIS_CLASS_IS_DEPRECATED = new StringIdImpl(3605, "this class is deprecated");
 -  public static final StringId DynamicRegionFactory_UNEXPECTED_EXCEPTION = new StringIdImpl(3606, "unexpected exception");
 -  public static final StringId CqAttributesFactory_CLONENOTSUPPORTEDEXCEPTION_THROWN_IN_CLASS_THAT_IMPLEMENTS_CLONEABLE = new StringIdImpl(3607, "CloneNotSupportedException thrown in class that implements cloneable");
 -  public static final StringId Support_ERROR_ASSERTION_FAILED_0 = new StringIdImpl(3608, "ERROR: Assertion failed: '' {0} ''");
 -  public static final StringId TypeUtils_EXPECTED_INSTANCE_OF_0_BUT_WAS_1 = new StringIdImpl(3609, "expected instance of  {0}  but was  {1}");
 -  public static final StringId SystemFailure_SINCE_THIS_IS_A_DEDICATED_CACHE_SERVER_AND_THE_JVM_HAS_BEEN_CORRUPTED_THIS_PROCESS_WILL_NOW_TERMINATE_PERMISSION_TO_CALL_SYSTEM_EXIT_INT_WAS_GIVEN_IN_THE_FOLLOWING_CONTEXT = new StringIdImpl(3610, "Since this is a dedicated cache server and the JVM has been corrupted, this process will now terminate. Permission to call System#exit(int) was given in the following context.");
 -  public static final StringId GatewayImpl_0_DID_NOT_WAIT_FOR_FAILOVER_COMPLETION_DUE_TO_INTERRUPTION = new StringIdImpl(3611, "{0}: did not wait for failover completion due to interruption.");
 -  public static final StringId GatewayImpl_0_AN_EXCEPTION_OCCURRED_WHILE_QUEUEING_1_TO_PERFORM_OPERATION_2_FOR_3 = new StringIdImpl(3612, "{0}: An Exception occurred while queueing {1} to perform operation {2} for {3}");
 -  public static final StringId GatewayImpl_THE_EVENT_QUEUE_SIZE_HAS_DROPPED_BELOW_THE_THRESHOLD_0 = new StringIdImpl(3613, "The event queue size has dropped below {0} events.");
 -  public static final StringId GatewayImpl_A_CACHECLOSEDEXCEPTION_OCCURRED_STOPPING_THE_DISPATCHER = new StringIdImpl(3614, "A CacheClosedException occurred. Stopping the dispatcher.");
 -  public static final StringId GatewayImpl_GATEWAY_FAILOVER_INITIATED_PROCESSING_0_UNPROCESSED_EVENTS = new StringIdImpl(3615, "Gateway Failover Initiated: Processing {0} unprocessed events.");
 -  public static final StringId GatewayImpl_A_BATCHEXCEPTION_OCCURRED_DURING_FAILOVER_PROCESSING_EVENT_0_THIS_MESSAGE_WILL_CONTINUE_TO_BE_PROCESSED = new StringIdImpl(3616, "A BatchException occurred during failover processing event {0}. This message will continue to be processed.");
 -  public static final StringId GatewayImpl_STOPPING_THE_DISPATCHER_BECAUSE_THE_FOLLOWING_EXCEPTION_OCCURRED_WHILE_PROCESSING_A_BATCH = new StringIdImpl(3617, "Stopping the dispatcher because the following exception occurred while processing a batch:");
 -  public static final StringId GatewayImpl_AN_EXCEPTION_OCCURRED_THE_DISPATCHER_WILL_CONTINUE = new StringIdImpl(3618, "An Exception occurred. The dispatcher will continue.");
 -  public static final StringId GatewayImpl_0_THE_UNPROCESSED_EVENTS_MAP_ALREADY_CONTAINED_AN_EVENT_FROM_THE_HUB_1_SO_IGNORING_NEW_EVENT_2 = new StringIdImpl(3619, "{0}: The secondary map already contained an event from hub {1} so ignoring new event {2}.");
 -  public static final StringId GatewayImpl_EVENT_FAILED_TO_BE_INITIALIZED_0 = new StringIdImpl(3620, "Event failed to be initialized: {0}");
 -  public static final StringId GatewayImpl_STOPPING_THE_DISPATCHER_BECAUSE_THE_FOLLOWING_GATEWAY_EXCEPTION_OCCURRED = new StringIdImpl(3621, "Stopping the dispatcher because the following gateway exception occurred.");
 -  public static final StringId GatewayImpl_0_THE_EVENT_QUEUE_SIZE_HAS_REACHED_THE_THRESHOLD_1 = new StringIdImpl(3622, "{0}: The event queue has reached {1} events. Processing will continue.");
 -  public static final StringId GatewayImpl_AN_INTERRUPTEDEXCEPTION_OCCURRED_BUT_THE_DISPATCHER_IS_NOT_STOPPED = new StringIdImpl(3623, "An InterruptedException occurred, but the dispatcher is not stopped.");
 -  public static final StringId GatewayImpl_AN_INTERRUPTEDEXCEPTION_OCCURRED_THE_THREAD_WILL_EXIT = new StringIdImpl(3624, "An InterruptedException occurred. The thread will exit.");
 -  public static final StringId GatewayImpl_A_BATCHEXCEPTION_OCCURRED_PROCESSING_EVENT_0 = new StringIdImpl(3625, "A BatchException occurred processing event {0}.");
 -  public static final StringId GatewayImpl_A_THREADINTERRUPTEDEXCEPTION_OCCURRED_STOPPING_THE_DISPATCHER = new StringIdImpl(3626, "A ThreadInterruptedException occurred. Stopping the dispatcher.");
 -  public static final StringId GatewayImpl_THE_FOLLOWING_CACHEEXCEPTION_OCCURRED_WHILE_REMOVING_0_EVENTS = new StringIdImpl(3627, "The following CacheException occurred while removing {0} events.");
 -  public static final StringId GatewayImpl_EVENT_DROPPED_DURING_FAILOVER_0 = new StringIdImpl(3628, "Event dropped during failover: {0}");
 -  public static final StringId GatewayImpl_0_DISPATCHER_STILL_ALIVE_EVEN_AFTER_JOIN_OF_5_SECONDS = new StringIdImpl(3629, "{0}:Dispatcher still alive even after join of 5 seconds.");
 -  public static final StringId GatewayImpl_0_INTERRUPTEDEXCEPTION_IN_JOINING_WITH_DISPATCHER_THREAD = new StringIdImpl(3630, "{0}:InterruptedException in joining with dispatcher thread.");
 -  public static final StringId GatewayImpl_THE_EVENT_BEING_PROCESSED_WHEN_THE_BATCHEXCEPTION_OCCURRED_WAS_0 = new StringIdImpl(3631, "The event being processed when the BatchException occurred was: {0}");
 -  public static final StringId AdminConfig_THIS_FILE_IS_GENERATED_BY_ADMINCONSOLE_EDIT_AS_YOU_WISH_BUT_IT_WILL_BE_OVERWRITTEN_IF_IT_IS_MODIFIED_IN_ADMINCONSOLE = new StringIdImpl(3632, "This file is generated by AdminConsole. Edit as you wish but it will be overwritten if it is modified in AdminConsole.");
 -  public static final StringId AdminConfig_MODIFIED_0 = new StringIdImpl(3633, "Modified {0}");
 -  public static final StringId DLockService_DISCONNECT_LISTENER_FOR_DISTRIBUTEDLOCKSERVICE = new StringIdImpl(3634, "Disconnect listener for DistributedLockService");
 -  public static final StringId DLockService_DISTRIBUTED_LOCKING_THREADS = new StringIdImpl(3635, "Distributed Locking Threads");
 -  // public static final StringId Oplog_BACKUP = new StringIdImpl(3636, "BACKUP");
 -  // public static final StringId Oplog_OVERFLOW = new StringIdImpl(3637, "OVERFLOW");
 -  public static final StringId DiskRegion_DISK_WRITERS = new StringIdImpl(3638, "Disk Writers");
 -  public static final StringId DiskRegion_ASYNCHRONOUS_DISK_WRITER_0 = new StringIdImpl(3639, "Asynchronous disk writer for region {0}");
 -  public static final StringId PartitionedRegion_SHUTDOWN_LISTENER_FOR_PARTITIONEDREGION = new StringIdImpl(3640, "Shutdown listener for PartitionedRegion");
 -  public static final StringId ExecuteFunction_CANNOT_SPECIFY_0_FOR_ONREGIONS_FUNCTION = new StringIdImpl(3641, "Cannot specify {0} for multi region function");
 -  public static final StringId PartitionedRegion_AN_EXCEPTION_WAS_CAUGHT_WHILE_REGISTERING_PARTITIONEDREGION_0_DUMPPRID_1 = new StringIdImpl(3642, "An exception was caught while registering PartitionedRegion \"{0}\". dumpPRId: {1}");
 -  public static final StringId Connection_COULD_NOT_START_READER_THREAD = new StringIdImpl(3643, "could not start reader thread");
 -  public static final StringId Connection_IDLE_CONNECTION_TIMED_OUT = new StringIdImpl(3644, "idle connection timed out");
 -  public static final StringId Connection_UNKNOWN = new StringIdImpl(3645, "unknown");
 -  public static final StringId Connection_HANDSHAKE_TIMED_OUT = new StringIdImpl(3646, "handshake timed out");
 -  public static final StringId Connection_INTERRUPTED = new StringIdImpl(3647, "interrupted");
 -  public static final StringId Connection_FAILED_HANDSHAKE = new StringIdImpl(3648, "failed handshake");
 -  public static final StringId Connection_FAILED_CONSTRUCTION = new StringIdImpl(3649, "failed construction");
 -  public static final StringId Connection_RUNNIOREADER_CAUGHT_CLOSED_CHANNEL = new StringIdImpl(3650, "runNioReader caught closed channel");
 -  public static final StringId Connection_RUNNIOREADER_CAUGHT_SHUTDOWN = new StringIdImpl(3651, "runNioReader caught shutdown");
 -  public static final StringId Connection_SOCKETCHANNEL_READ_RETURNED_EOF = new StringIdImpl(3652, "SocketChannel.read returned EOF");
 -  public static final StringId Connection_THREADINTERRUPTED_IN_CHANNEL_READ_0 = new StringIdImpl(3653, "ThreadInterrupted in channel read: {0}");
 -  public static final StringId Connection_CACHECLOSED_IN_CHANNEL_READ_0 = new StringIdImpl(3654, "CacheClosed in channel read: {0}");
 -  public static final StringId Connection_IOEXCEPTION_IN_CHANNEL_READ_0 = new StringIdImpl(3655, "IOException in channel read: {0}");
 -  public static final StringId Connection_P2P_MESSAGE_READER_FOR_0 = new StringIdImpl(3656, "P2P message reader for {0} on port {1}");
 -  public static final StringId Connection_CLOSEDCHANNELEXCEPTION_IN_CHANNEL_READ_0 = new StringIdImpl(3657, "ClosedChannelException in channel read: {0}");
 -  public static final StringId Connection_IOEXCEPTION_RECEIVED_0 = new StringIdImpl(3658, "IOException received: {0}");
 -  public static final StringId Connection_STREAM_READ_RETURNED_NONPOSITIVE_LENGTH = new StringIdImpl(3659, "Stream read returned non-positive length");
 -  public static final StringId Connection_CURRENT_THREAD_INTERRUPTED = new StringIdImpl(3660, "Current thread interrupted");
 -  public static final StringId ExecuteFunction_RESULTS_NOT_COLLECTED_IN_TIME_PROVIDED = new StringIdImpl(3661, "All results not received in time provided");
 -  public static final StringId Connection_NO_DISTRIBUTION_MANAGER = new StringIdImpl(3662, "no distribution manager");
 -  public static final StringId Connection_COULD_NOT_SEND_FORCE_DISCONNECT = new StringIdImpl(3663, "Could not send force disconnect");
 -  public static final StringId Connection_FORCE_DISCONNECT_TIMED_OUT = new StringIdImpl(3664, "Force disconnect timed out");
 -  public static final StringId Connection_P2P_PUSHER_IO_EXCEPTION_FOR_0 = new StringIdImpl(3665, "P2P pusher io exception for {0}");
 -  public static final StringId Connection_P2P_PUSHER_0_CAUGHT_CACHECLOSEDEXCEPTION_1 = new StringIdImpl(3666, "P2P pusher {0} caught CacheClosedException: {1}");
 -  public static final StringId Connection_P2P_PUSHER_0_CAUGHT_INTERRUPT_1 = new StringIdImpl(3667, "P2P pusher {0} caught interrupt: {1}");
 -  public static final StringId Connection_NONPOSITIVE_RETURN_FROM_CHANNEL_READ_OF_ACK = new StringIdImpl(3668, "non-positive return from channel read of ack");
 -  public static final StringId Connection_ACK_READ_IO_EXCEPTION_FOR_0 = new StringIdImpl(3669, "ack read io exception for {0}");
 -  public static final StringId Connection_CLASSNOTFOUND_DESERIALIZING_MESSAGE = new StringIdImpl(3670, "ClassNotFound deserializing message");
 -  public static final StringId Connection_FAILED_SENDING_HANDSHAKE_REPLY = new StringIdImpl(3671, "Failed sending handshake reply");
 -  public static final StringId ConnectionProxyImpl_ASYNCHRONOUS_REDUNDANCY_SATISFIER_THREAD_INVOKED_FROM_ENDPOINTDIED = new StringIdImpl(3672, "Asynchronous redundancy satisfier thread invoked from endpointDied");
 -  public static final StringId ConnectionProxyImpl_AFTER_REDUNDANT_SELECTION_HERE_ARE_THE_ENDPOINTS_0 = new StringIdImpl(3673, "After redundant selection, here are the endpoints: {0}");
 -  public static final StringId ConnectionProxyImpl_AFTER_A_FAILURE_ON_THE_PRIMARY_HERE_ARE_THE_ENDPOINTS_0 = new StringIdImpl(3674, "After a failure on the primary endpoint, here are the endpoints: {0}");
 -  public static final StringId ConnectionProxyImpl_AFTER_ADDING_AN_ENDPOINT_HERE_ARE_THE_ENDPOINTS_0 = new StringIdImpl(3675, "After adding an endpoint, here are the endpoints: {0}");
 -  public static final StringId ConnectionProxyImpl_UNREGISTERED_INTEREST_IN_0_ON_REGION_1_OF_TYPE_2_WITH_POLICY_3_BECAUSE_OF_4 = new StringIdImpl(3677, "unregistered interest in {0} on region {1} of type {2} with policy {3} because of {4}");
 -  public static final StringId JGroupMembershipManager_TCPIP_CONNECTIONS_CLOSED = new StringIdImpl(3678, "tcp/ip connections closed");
 -  public static final StringId JGroupMembershipManager_THE_MEMBER_WITH_ID_0_IS_NO_LONGER_IN_MY_OWN_VIEW_1 = new StringIdImpl(3679, "The Member with id {0}, is no longer in my own view, {1}");
 -  public static final StringId LicenseInfo_LICENSE_IS_VALID_UNTIL_0 = new StringIdImpl(3680, "License is valid until \"{0}\".");
 -  public static final StringId LicenseInfo_LICENSE_NEVER_EXPIRES = new StringIdImpl(3681, "License never expires.");
 -  public static final StringId LicenseInfo_LICENSE_IS_LIMITED_TO_NODES_0 = new StringIdImpl(3682, "License is limited to node(s) \"{0}\".");
 -  public static final StringId LicenseInfo_LICENSE_IS_VALID_ON_ANY_NODE = new StringIdImpl(3683, "License is valid on any node.");
 -  public static final StringId LicenseInfo_LICENSE_IS_LIMITED_TO_THIS_NATIVE_NODE_0 = new StringIdImpl(3684, "License is limited to this native node \"{0}\".");
 -  public static final StringId LicenseInfo_LICENSE_HAD_NO_NATIVE_NODE_LIMITS = new StringIdImpl(3685, "License had no native node limits.");
 -  public static final StringId LicenseInfo_LICENSE_LIMITS_DISTRIBUTED_SYSTEM_TO_0_MEMBERS = new StringIdImpl(3686, "License limits distributed system to \"{0}\" members.");
 -  public static final StringId LicenseInfo_LICENSE_IS_LIMITED_TO_0_CPUS = new StringIdImpl(3687, "License is limited to {0} cpus.");
 -  public static final StringId LicenseInfo_LICENSE_ALLOWS_FOR_HYPERTHREADED_CPUS = new StringIdImpl(3688, "License allows for hyperthreaded cpus.");
 -  public static final StringId LicenseInfo_ACTUAL_NUMBER_OF_PURCHASED_CPUS_IS_0 = new StringIdImpl(3689, "Actual number of purchased cpus is \"{0}\".");
 -  public static final StringId LicenseInfo_LICENSE_ALLOWS_AN_UNLIMITED_NUMBER_OF_CPUS = new StringIdImpl(3690, "License allows an unlimited number of cpus.");
 -  public static final StringId LicenseInfo_NO_LIMIT_FOR_KEY_0 = new StringIdImpl(3691, "No limit for key \"{0}\"");
 -  public static final StringId LicenseInfo_DETERMINED_BY_GEMSTONE = new StringIdImpl(3692, "<determined by GemStone>");
 -  public static final StringId LicenseInfo_PLEASE_SUPPLY_A_CUSTOMER_NAME_HERE = new StringIdImpl(3693, "<Please supply a customer name here>");
 -  public static final StringId LicenseInfo_DATA_NEEDED_TO_OBTAIN_LICENSE_FOR_CURRENT_MACHINE = new StringIdImpl(3694, "Data Needed to Obtain License for current machine.");
 -  public static final StringId LicenseInfo_CONTENTS_OF_0 = new StringIdImpl(3695, "-------------------- Contents of {0} --------------------");
 -  public static final StringId LicenseInfo_COULD_NOT_FIND_DEFAULT_LICENSE_FILE_0 = new StringIdImpl(3696, "Could not find default license file: {0}");
 -  public static final StringId LicenseInfo_TROUBLE_READING_LICENSE_FILE_0 = new StringIdImpl(3697, "Trouble reading license file: {0}");
 -  public static final StringId LicenseInfo_LICENSE_SIGNATURE_IS_VALID = new StringIdImpl(3698, "License signature is valid.");
 -  public static final StringId LicenseInfo_THIS_LICENSE_IS_INVALID_0 = new StringIdImpl(3699, "This license is invalid. {0}");
 -  public static final StringId GemFireVersion_COULD_NOT_FIND_GEMFIRE_JAR = new StringIdImpl(3700, "<Could not find gemfire.jar>");
 -  public static final StringId GemFireVersion_COULD_NOT_FIND_RESOURCE_COM_GEMSTONE_GEMFIRE_INTERNAL_0 = new StringIdImpl(3701, "<Could not find resource com/gemstone/gemfire/internal/{0}>");
 -  public static final StringId GemFireVersion_COULD_NOT_READ_PROPERTIES_FROM_RESOURCE_COM_GEMSTONE_GEMFIRE_INTERNAL_0_BECAUSE_1 = new StringIdImpl(3702, "<Could not read properties from resource com/gemstone/gemfire/internal/{0} because: {1}>");
 -  public static final StringId GemFireVersion_MISSING_PROPERTY_0_FROM_RESOURCE_COM_GEMSTONE_GEMFIRE_INTERNAL_1 = new StringIdImpl(3703, "<Missing property {0} from resource com/gemstone/gemfire/internal/{1}>");
 -  public static final StringId GemFireVersion_RUNNING_ON_0 = new StringIdImpl(3704, "Running on: {0}");
 -  public static final StringId GemFireVersion_WARNING_EXPECTED_JAVA_VERSION_0 = new StringIdImpl(3705, "Warning expected java version: {0}");
 -  public static final StringId GemFireVersion_WARNING_EXPECTED_NATIVE_VERSION_0 = new StringIdImpl(3706, "Warning expected native version: {0}");
 -  public static final StringId GemFireVersion_WARNING_FAILED_TO_READ_0_BECAUSE_1 = new StringIdImpl(3707, "Warning failed to read \"{0}\" because {1}.");
 -  public static final StringId GemFireVersion_WARNING_COULD_NOT_FIND_0 = new StringIdImpl(3708, "Warning could not find \"{0}\".");
 -  public static final StringId GemFireVersion_WARNING_COULD_NOT_DETERMINE_THE_PRODUCTS_LIB_DIRECTORY = new StringIdImpl(3709, "Warning could not determine the product''s lib directory.");
 -  public static final StringId LocalRegion_THE_CACHE_IS_NOT_AVAILABLE = new StringIdImpl(3710, "The cache is not available");
 -  public static final StringId LocalRegion_THE_DISTRIBUTED_SYSTEM_IS_NOT_AVAILABLE = new StringIdImpl(3711, "The distributed system is not available.");
 -  public static final StringId GemFireCache_NORMAL_DISCONNECT = new StringIdImpl(3712, "Normal disconnect");
 -  public static final StringId GlobalTransaction_GLOBALTRANSACTION_ADDTRANSACTION_CANNOT_ADD_A_NULL_TRANSACTION = new StringIdImpl(3713, "GlobalTransaction::addTransaction::Cannot add a null Transaction");
 -  public static final StringId GlobalTransaction_GLOBALTRANSACTION_COMMIT_ERROR_IN_COMMITTING_BUT_TRANSACTION_COULD_NOT_BE_ROLLED_BACK_DUE_TO_EXCEPTION_0 = new StringIdImpl(3714, "GlobalTransaction::commit::Error in committing, but transaction could not be rolled back due to exception: {0}");
 -  public static final StringId GlobalTransaction_GLOBALTRANSACTION_COMMIT_ERROR_IN_COMMITTING_THE_TRANSACTION_TRANSACTION_ROLLED_BACK_EXCEPTION_0_1 = new StringIdImpl(3715, "GlobalTransaction::commit:Error in committing the transaction. Transaction rolled back.Exception = {0} {1}");
 -  public static final StringId GlobalTransaction_GLOBALTRANSACTION_ROLLBACK_ROLLBACK_NOT_SUCCESSFUL_DUE_TO_EXCEPTION_0_1 = new StringIdImpl(3716, "GlobalTransaction::rollback:Rollback not successful due to exception {0} {1}");
 -  public static final StringId GlobalTransaction_GLOBALTRANSACTION_ENLISTRESOURCE_CANNOT_ENLIST_A_RESOURCE_TO_A_TRANSACTION_WHICH_IS_NOT_ACTIVE = new StringIdImpl(3717, "GlobalTransaction::enlistResource::Cannot enlist a resource to a transaction which is not active");
 -  public static final StringId GlobalTransaction_GLOBALTRANSACTION_ENLISTRESOURCE_EXCEPTION_OCCURED_IN_TRYING_TO_SET_XARESOURCE_TIMEOUT_DUE_TO_0_ERROR_CODE_1 = new StringIdImpl(3718, "GlobalTransaction::enlistResource:Exception occurred in trying to set XAResource timeout due to {0} Error Code = {1}");
 -  public static final StringId GlobalTransaction_ERROR_WHILE_DELISTING_XARESOURCE_0_1 = new StringIdImpl(3719, "error while delisting XAResource {0} {1}");
 -  public static final StringId GlobalTransaction_GLOBATRANSACTION_RESUME_RESUME_NOT_SUCCESFUL_DUE_TO_0 = new StringIdImpl(3720, "GlobaTransaction::resume:Resume not succesful due to {0}");
 -  public static final StringId GlobalTransaction_EXCEPTION_OCCURED_WHILE_TRYING_TO_SET_THE_XARESOURCE_TIMEOUT_DUE_TO_0_ERROR_CODE_1 = new StringIdImpl(3721, "Exception occurred while trying to set the XAResource TimeOut due to {0} Error code = {1}");
 -  public static final StringId SmHelper_NATIVE_CODE_UNAVAILABLE = new StringIdImpl(3722, "native code unavailable");
 -  public static final StringId SystemAdmin_WAITING_5_SECONDS_FOR_LOCATOR_PROCESS_TO_TERMINATE = new StringIdImpl(3723, "Waiting 5 seconds for locator process to terminate...");
 -  public static final StringId SystemAdmin_WAITING_FOR_LOCATOR_PROCESS_WITH_PID_0_TO_TERMINATE = new StringIdImpl(3724, "Waiting for locator process, with pid {0,number,#} to terminate...");
 -  public static final StringId SystemAdmin_LOCATOR_PROCESS_HAS_TERMINATED = new StringIdImpl(3725, "Locator process has terminated.");
 -  public static final StringId SystemAdmin_LOCATOR_IN_0_WAS_KILLED_WHILE_IT_WAS_1_LOCATOR_PROCESS_ID_WAS_2 = new StringIdImpl(3726, "Locator in \"{0}\" was killed while it was {1}. Locator process id was {2}.");
 -  public static final StringId SystemAdmin_LOCATOR_IN_0_IS_1_LOCATOR_PROCESS_ID_IS_2 = new StringIdImpl(3727, "Locator in \"{0}\" is {1}. Locator process id is {2}.");
 -  public static final StringId SystemAdmin_LOCATOR_IN_0_IS_STOPPED = new StringIdImpl(3728, "Locator in \"{0}\" is stopped.");
 -  public static final StringId SystemAdmin_LOCATOR_IN_0_IS_STARTING = new StringIdImpl(3729, "Locator in \"{0}\" is starting.");
 -  public static final StringId SystemAdmin_CLEANED_UP_ARTIFACTS_LEFT_BY_THE_PREVIOUS_KILLED_LOCATOR = new StringIdImpl(3730, "Cleaned up artifacts left by the previous killed locator.");
 -  public static final StringId SystemAdmin_LOG_FILE_0_DOES_NOT_EXIST = new StringIdImpl(3731, "Log file \"{0}\" does not exist.");
 -  public static final StringId SystemAdmin_AN_IOEXCEPTION_WAS_THROWN_WHILE_TAILING_0 = new StringIdImpl(3732, "An IOException was thrown while tailing \"{0}\"\n");
 -  public static final StringId SystemAdmin_MERGING_THE_FOLLOWING_LOG_FILES = new StringIdImpl(3733, "Merging the following log files:");
 -  public static final StringId SystemAdmin_COMPLETED_MERGE_OF_0_LOGS_TO_1 = new StringIdImpl(3734, "Completed merge of {0} logs to \"{1}\".");
 -  public static final StringId SystemAdmin_WARNING_NO_STATS_MATCHED_0 = new StringIdImpl(3735, "[warning] No stats matched \"{0}\".");
 -  public static final StringId SystemAdmin_ERROR_OPERATION_0_FAILED_BECAUSE_1 = new StringIdImpl(3736, "ERROR: Operation \"{0}\" failed because: {1}.");
 -  public static final StringId SystemAdmin_ERROR_UNKNOWN_COMMAND_0 = new StringIdImpl(3737, "ERROR: Unknown command \"{0}\".");
 -  public static final StringId SystemAdmin_INFO_FOUND_0_MATCHES_FOR_1 = new StringIdImpl(3738, "[info] Found {0} instances matching \"{1}\":");
 -  public static final StringId SystemAdmin_THIS_PROGRAM_ALLOWS_GEMFIRE_TO_BE_MANAGED_FROM_THE_COMMAND_LINE_IT_EXPECTS_A_COMMAND_TO_EXECUTE_SEE_THE_HELP_TOPIC_0_FOR_A_SUMMARY_OF_SUPPORTED_OPTIONS_SEE_THE_HELP_TOPIC_1_FOR_A_CONCISE_DESCRIPTION_OF_COMMAND_LINE_SYNTAX_SEE_THE_HELP_TOPIC_2_FOR_A_DESCRIPTION_OF_SYSTEM_CONFIGURATION_SEE_THE_HELP_TOPIC_3_FOR_HELP_ON_A_SPECIFIC_COMMAND_USE_THE_4_OPTION_WITH_THE_COMMAND_NAME = new StringIdImpl(3739, "This program allows GemFire to be managed from the command line. It expects a command to execute.\nSee the help topic \"{0}\". For a summary of supported options see the help topic \"{1}\".\nFor a concise description of command line syntax see the help topic \"{2}\".\nFor a description of system configuration see the help topic \"{3}\".\nFor help on a specific command use the \"{4}\" option with the command name.");
 -  public static final StringId SystemAdmin_ALL_COMMAND_LINE_OPTIONS_START_WITH_A_AND_ARE_NOT_REQUIRED_EACH_OPTION_HAS_A_DEFAULT_THAT_WILL_BE_USED_WHEN_ITS_NOT_SPECIFIED_OPTIONS_THAT_TAKE_AN_ARGUMENT_ALWAYS_USE_A_SINGLE_CHARACTER_WITH_NO_SPACES_TO_DELIMIT_WHERE_THE_OPTION_NAME_ENDS_AND_THE_ARGUMENT_BEGINS_OPTIONS_THAT_PRECEDE_THE_COMMAND_WORD_CAN_BE_USED_WITH_ANY_COMMAND_AND_ARE_ALSO_PERMITTED_TO_FOLLOW_THE_COMMAND_WORD = new StringIdImpl(3740, "All command line options start with a \"-\" and are not required.\nEach option has a default that will be used when its not specified.\nOptions that take an argument always use a single \"=\" character, with no spaces, to delimit where the option name ends and the argument begins.\nOptions that precede the command word can be used with any command and are also permitted to follow the command word.");
 -  public static final StringId SystemAdmin_NO_HELP_FOR_OPTION_0 = new StringIdImpl(3741, "no help for option \"{0}]\"");
 -  public static final StringId SystemAdmin_EXPLAINATION_OF_COMMAND_OPTIONS = new StringIdImpl(3742, "The following synax is used in the usage strings:\n\"[]\" designate an optional item\n\"()\" are used to group items\n\"<>\" designate non-literal text. Used to designate logical items\n\"*\" suffix means zero or more of the previous item\n\"|\" means the item to the left or right is required");
 -  public static final StringId SystemAdmin_ERROR_INVALID_HELP_TOPIC_0 = new StringIdImpl(3743, "ERROR: Invalid help topic \"{0}\".");
 -  public static final StringId SystemAdmin_USAGE = new StringIdImpl(3744, "Usage");
 -  public static final StringId SystemAdmin_ERROR_INVALID_COMMAND_0 = new StringIdImpl(3745, "ERROR: Invalid command \"{0}\".");
 -  public static final StringId SystemAdmin_GEMFIRE_HELP = new StringIdImpl(3746, "gemfire requires one of the following command strings:\n{0}\nFor additional help on a command specify it along with the \"{1}\" option.\nThe \"{2}\" option causes gemfire to print out extra information when it fails.\nThe \"{1}\" and \"{3}\" are synonyms that cause gemfire to print out help information instead of performing a task.\nThe \"{4}\" option quiets gemfire down by suppressing extra messages.\nThe \"{5}\" option passes <vmOpt> to the java vm''s command line.");
 -  public static final StringId SystemAdmin_VERSION_HELP = new StringIdImpl(3747, "Prints GemFire product version information.");
 -  public static final StringId SystemAdmin_LICENSE_HELP_0 = new StringIdImpl(3748, "Prints any current GemFire licenses and information needed to obtain a new license.\nThe \"{0}\" option can be used to specify the license file to print.");
 -  public static final StringId SystemAdmin_HELP_HELP = new StringIdImpl(3749, "Prints information on how to use this executable.\nIf an optional help topic is specified then more detailed help is printed.");
 -  public static final StringId SystemAdmin_STATS_HELP_PART_A = new StringIdImpl(3750, "Prints statistic values from a statistic archive\nBy default all statistics are printed.\nThe statSpec arguments can be used to print individual resources or a specific statistic.\nThe format of a statSpec is: an optional combine operator, followed by an optional instanceId, followed by an optional typeId, followed by an optional statId.\nA combine operator can be \"{0}\" to combine all matches in the same file, or \"{1}\" to combine all matches across all files.\nAn instanceId must be the name or id of a resource.\nA typeId is a \"{2}\" followed by the name of a resource type.\nA statId is a \"{3}\" followed by the name of a statistic.\nA typeId or instanceId with no statId prints out all the matching resources and all their statistics.\nA typeId or instanceId with a statId prints out just the named statistic on the matching resources.\nA statId with no typeId or instanceId matches all statistic
 s with that name.\nThe \"{4}\" option causes statistic descriptions to also be printed.\nThe \"{5}\" option, in conjunction with \"{6}\", causes the printed statistics to all be raw, unfiltered, values.\nThe \"{7}\" option, in conjunction with \"{6}\", causes the printed statistics to be the rate of change, per second, of the raw values.\nThe \"{8}\" option, in conjunction with \"{6}\", causes the printed statistics to be the rate of change, per sample, of the raw values.\nThe \"{9}\" option, in conjunction with \"{6}\", causes statistics whose values are all zero to not be printed.");
 -  public static final StringId SystemAdmin_STATS_HELP_PART_B = new StringIdImpl(3751, "The \"{0}\" option, in conjunction with \"{1}\", causes statistics samples taken before this time to be ignored. The argument format must match \"{2}\".\nThe \"{3}\" option, in conjunction with \"{1}\", causes statistics samples taken after this time to be ignored. The argument format must match \"{2}\".\nThe \"{1}\" option causes the data to come from an archive file.");
 -  public static final StringId SystemAdmin_ENCRYPTS_A_PASSWORD_FOR_USE_IN_CACHE_XML_DATA_SOURCE_CONFIGURATION = new StringIdImpl(3752, "Encrypts a password for use in cache.xml data source configuration.");
 -  public static final StringId SystemAdmin_START_LOCATOR_HELP = new StringIdImpl(3753, "Starts a locator.\nThe \"{0}\" option specifies the port the locator will listen on. It defaults to \"{1}\"\nThe \"{2}\" option specifies the address the locator will listen on. It defaults to listening on all local addresses.\nThe \"{3}\" option can be used to specify the directory the locator will run in.\nThe \"{4}\" option can be used to specify the gemfire.properties file for configuring the locator''s distributed system.  The file''s path should be absolute, or relative to the locator''s directory ({3})\nThe \"{5}\" option can be used to specify whether peer locator service should be enabled. True (the default) will enable the service.\nThe \"{6}\" option can be used to specify whether server locator service should be enabled. True (the default) will enable the service.\nThe \"{7}\" option can be used to specify a host name or ip address that will be sent to clients so they can connect to 
 this locator. The default is to use the address the locator is listening on.\nThe \"{8}\" option can be used to set system properties for the locator VM\nThe \"{9}\" option can be used to set vendor-specific VM options and is usually used to increase the size of the locator VM when using multicast.\n");
 -   public static final StringId SystemAdmin_STOP_LOCATOR_HELP = new StringIdImpl(3754, "Stops a locator.\nThe \"{0}\" option specifies the port the locator is listening on. It defaults to \"{1}\"\nThe \"{2}\" option specifies the address the locator is listening on. It defaults to the local host''s address.\nThe \"{3}\" option can be used to specify the directory the locator is running in.");
 -  public static final StringId SystemAdmin_STATUS_LOCATOR_HELP = new StringIdImpl(3755, "Prints the status of a locator. The status string will one of the following:\n{0}\nThe \"{1}\" option can be used to specify the directory of the locator whose status is desired.");
 -  public static final StringId SystemAdmin_INFO_LOCATOR_HELP = new StringIdImpl(3756, "Prints information on a locator.\nThe information includes the process id of the locator, if the product is not running in PureJava mode.\nThe \"{0}\" option can be used to specify the directory of the locator whose information is desired.");
 -  public static final StringId SystemAdmin_TAIL_LOCATOR_HELP = new StringIdImpl(3757, "Prints the last 64K bytes of the locator''s log file.\nThe \"{0}\" option can be used to specify the directory of the locator whose information is desired.");
 -  public static final StringId SystemAdmin_MERGE_LOGS = new StringIdImpl(3758, "Merges multiple logs files into a single log.\nThe \"{0}\" option can be used to specify the file to write the merged log to. The default is stdout.");
 -  public static final StringId SystemAdmin_CAUSES_GEMFIRE_TO_WRITE_OUTPUT_TO_THE_SPECIFIED_FILE_THE_FILE_IS_OVERWRITTEN_IF_IT_ALREADY_EXISTS = new StringIdImpl(3759, "Causes gemfire to write output to the specified file. The file is overwritten if it already exists.");
 -  public static final StringId SystemAdmin_CAUSES_GEMFIRE_TO_PRINT_OUT_EXTRA_INFORMATION_WHEN_IT_FAILS_THIS_OPTION_IS_SUPPORTED_BY_ALL_COMMANDS = new StringIdImpl(3760, "Causes gemfire to print out extra information when it fails. This option is supported by all commands.");
 -  public static final StringId SystemAdmin_CAUSES_GEMFIRE_TO_PRINT_DETAILED_INFORMATION_WITH_THE_0_COMMAND_IT_MEANS_STATISTIC_DESCRIPTIONS = new StringIdImpl(3761, "Causes gemfire to print detailed information.  With the \"{0}\" command it means statistic descriptions.");
 -  public static final StringId SystemAdmin_CAUSES_GEMFIRE_0_COMMAND_TO_PRINT_UNFILTERED_RAW_STATISTIC_VALUES_THIS_IS_THE_DEFAULT_FOR_NONCOUNTER_STATISTICS = new StringIdImpl(3762, "Causes gemfire \"{0}\" command to print unfiltered, raw, statistic values. This is the default for non-counter statistics.");
 -  public static final StringId SystemAdmin_CAUSES_GEMFIRE_0_COMMAND_TO_PRINT_THE_RATE_OF_CHANGE_PER_SECOND_FOR_STATISTIC_VALUES_THIS_IS_THE_DEFAULT_FOR_COUNTER_STATISTICS = new StringIdImpl(3763, "Causes gemfire \"{0}\" command to print the rate of change, per second, for statistic values. This is the default for counter statistics.");
 -  public static final StringId SystemAdmin_CAUSES_GEMFIRE_0_COMMAND_TO_PRINT_THE_RATE_OF_CHANGE_PER_SAMPLE_FOR_STATISTIC_VALUES = new StringIdImpl(3764, "Causes gemfire \"{0}\" command to print the rate of change, per sample, for statistic values.");
 -  public static final StringId SystemAdmin_CAUSES_GEMFIRE_0_COMMAND_TO_NOT_PRINT_STATISTICS_WHOSE_VALUES_ARE_ALL_ZERO = new StringIdImpl(3765, "Causes gemfire \"{0}\" command to not print statistics whose values are all zero.");
 -  public static final StringId SystemAdmin_USED_TO_SPECIFY_A_NONDEFAULT_PORT_WHEN_STARTING_OR_STOPPING_A_LOCATOR = new StringIdImpl(3766, "Used to specify a non-default port when starting or stopping a locator.");
 -  public static final StringId SystemAdmin_USED_TO_SPECIFY_A_SPECIFIC_IP_ADDRESS_TO_LISTEN_ON_WHEN_STARTING_OR_STOPPING_A_LOCATOR = new StringIdImpl(3767, "Used to specify a specific IP address to listen on when starting or stopping a locator.");
 -  public static final StringId SystemAdmin_THE_ARGUMENT_IS_THE_LICENSE_FILE_WHOSE_CONTENTS_WILL_BE_PRINTED_DEFAULTS_TO_0 = new StringIdImpl(3768, "The argument is the license file whose contents will be printed. Defaults to \"{0}\"");
 -  public static final StringId SystemAdmin_THE_ARGUMENT_IS_THE_STATISTIC_ARCHIVE_FILE_THE_0_COMMAND_SHOULD_READ = new StringIdImpl(3769, "The argument is the statistic archive file the \"{0}\" command should read.");
 -  public static final StringId SystemAdmin_CAUSES_GEMFIRE_TO_PRINT_OUT_INFORMATION_INSTEAD_OF_PERFORMING_THE_COMMAND_THIS_OPTION_IS_SUPPORTED_BY_ALL_COMMANDS = new StringIdImpl(3770, "Causes GemFire to print out information instead of performing the command. This option is supported by all commands.");
 -  public static final StringId SystemAdmin_TURNS_ON_QUIET_MODE_THIS_OPTION_IS_SUPPORTED_BY_ALL_COMMANDS = new StringIdImpl(3771, "Turns on quiet mode. This option is supported by all commands.");
 -  public static final StringId SystemAdmin_CAUSES_THE_0_COMMAND_TO_IGNORE_STATISTICS_SAMPLES_TAKEN_BEFORE_THIS_TIME_THE_ARGUMENT_FORMAT_MUST_MATCH_1 = new StringIdImpl(3772, "Causes the \"{0}\" command to ignore statistics samples taken before this time. The argument format must match \"{1}\".");
 -  public static final StringId SystemAdmin_CAUSES_THE_0_COMMAND_TO_IGNORE_STATISTICS_SAMPLES_TAKEN_AFTER_THIS_TIME_THE_ARGUMENT_FORMAT_MUST_MATCH_1 = new StringIdImpl(3773, "Causes the \"{0}\" command to ignore statistics samples taken after this time. The argument format must match \"{1}\".");
 -  public static final StringId SystemAdmin_DIR_ARGUMENT_HELP = new StringIdImpl(3774, "The argument is the system directory the command should operate on.\nIf the argument is empty then a default system directory will be search for.\nHowever the search will not include the \"{0}\" file.\nBy default if a command needs a system directory, and one is not specified, then a search is done. If a \"{0}\" file can be located then \"{1}\" property from that file is used. Otherwise if the \"{2}\" environment variable is set to a directory that contains a subdirectory named \"{3}\" then that directory is used.\nThe property file is search for in the following locations:\n1. The current working directory.\n2. The user''s home directory.\n3. The class path.\nAll commands except \"{4}\", and \"{5}\" use the system directory.");
 -  public static final StringId SystemAdmin_SETS_A_JAVA_SYSTEM_PROPERTY_IN_THE_LOCATOR_VM_USED_MOST_OFTEN_FOR_CONFIGURING_SSL_COMMUNICATION = new StringIdImpl(3775, "Sets a Java system property in the locator VM.  Used most often for configuring SSL communication.");
 -  public static final StringId SystemAdmin_SETS_A_JAVA_VM_X_SETTING_IN_THE_LOCATOR_VM_USED_MOST_OFTEN_FOR_INCREASING_THE_SIZE_OF_THE_VIRTUAL_MACHINE = new StringIdImpl(3776, "Sets a Java VM X setting in the locator VM.  Used most often for increasing the size of the virtual machine.");
 -  public static final StringId SystemAdmin_HELP = new StringIdImpl(3777, "help");
 -  public static final StringId SystemAdmin_ERROR_UNKNOWN_OPTION_0 = new StringIdImpl(3778, "ERROR: Unknown option \"{0}\".");
 -  public static final StringId SystemAdmin_ERROR_WRONG_NUMBER_OF_COMMAND_LINE_ARGS = new StringIdImpl(3779, "ERROR: Wrong number of command line args.");
 -  public static final StringId SystemAdmin_ERROR_UNEXPECTED_COMMAND_LINE_ARGUMENTS_0 = new StringIdImpl(3780, "ERROR: unexpected command line arguments: \"{0}\".");
 -  public static final StringId SystemAdmin_GEMFIRE_PRODUCT_DIRECTORY_0 = new StringIdImpl(3781, "GemFire product directory: {0}");
 -  public static final StringId SystemAdmin_LOCATOR_START_COMPLETE = new StringIdImpl(3782, "Locator start complete.");
 -  public static final StringId SystemAdmin_LOCATOR_STOP_COMPLETE = new StringIdImpl(3783, "Locator stop complete.");
 -  public static final StringId SystemAdmin_ERROR_EXPECTED_AT_LEAST_ONE_LOG_FILE_TO_MERGE = new StringIdImpl(3784, "ERROR: expected at least one log file to merge.");
 -  public static final StringId AgentConfigImpl_USING_DEFAULT_CONFIGURATION_BECAUSE_PROPERTY_FILE_WAS_FOUND = new StringIdImpl(3785, "Using default configuration because property file was not found.");
 -  public static final StringId AgentConfigImpl_CONFIGURATION_LOADED_FROM_0 = new StringIdImpl(3786, "Configuration loaded from: {0}.");
 -  public static final StringId AgentConfigImpl_NAME_OF_THE_AGENTS_LOG_FILE = new StringIdImpl(3787, "Name of the agent''s log file");
 -  public static final StringId AgentConfigImpl_MINIMUM_LEVEL_OF_LOGGING_PERFORMED_BY_AGENT = new StringIdImpl(3788, "Minimum level of logging performed by agent. Valid values are: all, finest, finer, fine, config, info, warning, error, severe and none.");
 -  public static final StringId AgentConfigImpl_WHETHER_THE_AGENT_SHOULD_PRINT_DEBUGGING_INFORMATION = new StringIdImpl(3789, "Whether the agent should print debugging information");
 -  public static final StringId AgentConfigImpl_LIMIT_IN_MEGABYTES_OF_HOW_MUCH_DISK_SPACE_CAN_BE_CONSUMED_BY_OLD_INACTIVE_LOG_FILES = new StringIdImpl(3790, "Limit, in megabytes, of how much disk space can be consumed by old inactive log files. This value (in megabytes) should be in the range: 0-1000000.");
 -  public static final StringId AgentConfigImpl_LIMIT_IN_MEGABYTES_OF_HOW_LARGE_THE_CURRENT_STATISTIC_ARCHIVE_FILE_CAN_GROW_BEFORE_IT_IS_CLOSED_AND_ARCHIVAL_ROLLS_ON_TO_A_NEW_FILE = new StringIdImpl(3791, "Limit, in megabytes, of how large the current log file can grow before it is closed and log rolls on to a new file. This value (in megabytes) should be in the range: 0-1000000.");
 -  public static final StringId AgentConfigImpl_MULTICAST_PORT_USED_TO_CONNECT_TO_DISTRIBUTED_SYSTEM = new StringIdImpl(3792, "Multicast port used to connect to distributed system. To use IP multicast, you must also define mcast-address. The value must be in the range: 0-65535.");
 -  public static final StringId AgentConfigImpl_MULTICAST_ADDRESS_USED_TO_CONNECT_TO_DISTRIBUTED_SYSTEM = new StringIdImpl(3793, "Multicast address used to connect to distributed system. To use multicast, you must also define mcast-port, the IP port.");
 -  public static final StringId AgentConfigImpl_ADDRESSES_OF_THE_LOCATORS_OF_THE_DISTRIBUTED_SYSTEM = new StringIdImpl(3794, "A comma-separated list of address(es) of the locator(s) in the distributed system in host[port] form. E.g. locators=host1[port1],host2[port2],...,hostn[portn]");
 -  public static final StringId AgentConfigImpl_XML_CONFIGURATION_FILE_FOR_MANAGED_ENTITIES = new StringIdImpl(3795, "The name of an XML file that specifies the configuration for the managed entity administered by the Distributed System. The XML file must conform to the dtd - doc-files/ds5_0.dtd.");
 -  public static final StringId AgentConfigImpl_WILL_THE_AGENT_AUTOMATICALLY_CONNECT_TO_THE_DISTRIBUTED_SYSTEM = new StringIdImpl(3796, "Whether the JMX agent will connect ''automatically'' to the distributed system that it is configured to monitor.");
 -  public static final StringId AgentConfigImpl_COMMAND_PREFIX_USED_FOR_LAUNCHING_MEMBERS_OF_THE_DISTRIBUTED_SYSTEM = new StringIdImpl(3797, "Command prefix used for launching members of the distributed system");
 -  public static final StringId AgentConfigImpl_WILL_THE_AGENT_START_THE_HTTP_JMX_ADAPTER = new StringIdImpl(3798, "Whether the HTTP adapter is enabled in the JMX agent.");
 -  public static final StringId AgentConfigImpl_BIND_ADDRESS_OF_HTTP_ADAPTERS_SOCKETS = new StringIdImpl(3799, "Bind address of HTTP adapter''s sockets");
 -  public static final StringId AgentConfigImpl_THE_PORT_ON_WHICH_THE_HTTP_ADAPTER_WILL_BE_STARTED = new StringIdImpl(3800, "The port on which the HTTP adapter will be started. This value should be in the range: 0-65535.");
 -  public static final StringId AgentConfigImpl_WILL_THE_AGENT_START_THE_RMI_JMX_ADAPTER = new StringIdImpl(3801, "Whether the RMI JMX adapter is enabled.");
 -  public static final StringId AgentConfigImpl_WILL_THE_AGENT_HOST_AN_RMI_REGISTRY = new StringIdImpl(3802, "Whether the JMX agent should start RMI registry. Alternatively, a registry outside of the JMX agent VM can be used.");
 -  public static final StringId AgentConfigImpl_BIND_ADDRESS_OF_RMI_ADAPTERS_SOCKETS = new StringIdImpl(3803, "Bind address of RMI adapter''s sockets");
 -  public static final StringId AgentConfigImpl_THE_PORT_ON_WHICH_TO_CONTACT_THE_RMI_REGISTER = new StringIdImpl(3804, "The port on which to contact the RMI registry. The value must be in the range: 0-65535.");
 -  public static final StringId AgentConfigImpl_WILL_THE_AGENT_START_THE_SNMP_JMX_ADAPTER = new StringIdImpl(3805, "Whether the SNMP JMX adapter will be enabled.");
 -  public static final StringId AgentConfigImpl_BIND_ADDRESS_OF_SNMP_ADAPTERS_SOCKETS = new StringIdImpl(3806, "Bind address of SNMP adapter''s sockets");
 -  public static final StringId AgentConfigImpl_THE_DIRECTORY_IN_WHICH_SNMP_CONFIGURATION_RESIDES = new StringIdImpl(3807, "The directory in which SNMP configuration resides");
 -  public static final StringId AgentConfigImpl_WILL_THE_AGENT_COMMUNICATE_USING_SSL = new StringIdImpl(3808, "Whether the JMX Agent will use the SSL protocol for communication.");
 -  public static final StringId AgentConfigImpl_THE_SSL_PROTOCOLS_USED_BY_THE_AGENT = new StringIdImpl(3809, "The space-separated list of the SSL protocols to be used when connecting to the JMX agent.");
 -  public static final StringId AgentConfigImpl_THE_SSL_CIPHERS_USED_BY_THE_AGENT = new StringIdImpl(3810, "The space-separated list of the SSL ciphers to be used when connecting to the JMX Agent.");
 -  public static final StringId AgentConfigImpl_WILL_THE_AGENT_REQUIRE_SSL_AUTHENTICATION = new StringIdImpl(3811, "Whether or not SSL connections to the RMI adapter require authentication. If true, needs client authentication for RMI and other non-HTTP connectors/adaptors.");
 -  public static final StringId AgentConfigImpl_WILL_THE_HTTP_ADAPTER_REQUIRE_SSL_AUTHENTICATION = new StringIdImpl(3812, "Whether SSL connections to the HTTP adapter will need authentication.");
 -  public static final StringId AgentConfigImpl_WILL_THE_HTTP_JMX_ADAPTER_USE_HTTP_AUTHENTICATION = new StringIdImpl(3813, "Whether the HTTP adapter will use HTTP authentication.");
 -  public static final StringId AgentConfigImpl_THE_USER_NAME_FOR_AUTHENTICATION_IN_THE_HTTP_JMX_ADAPTER = new StringIdImpl(3814, "The user name for authentication in the HTTP JMX adapter");
 -  public static final StringId AgentConfigImpl_THE_PASSWORD_FOR_AUTHENTICATION_IN_THE_HTTP_JMX_ADAPTER = new StringIdImpl(3815, "The password for authentication in the HTTP JMX adapter");
 -  public static final StringId AgentConfigImpl_DOES_THE_DISTRIBUTED_SYSTEM_COMMUNICATE_USING_SSL = new StringIdImpl(3816, "Whether to use the SSL protocol for communication between members of the admin distributed system. If set to true, locators should be used.");
 -  public static final StringId AgentConfigImpl_SSL_PROTOCOLS_USED_TO_COMMUNICATE_WITH_DISTRIBUTED_SYSTEM = new StringIdImpl(3817, "A space-separated list of the SSL protocols used to communicate with distributed system.");
 -  public static final StringId AgentConfigImpl_SSL_CIPHERS_USED_TO_COMMUNICATE_WITH_DISTRIBUTED_SYSTEM = new StringIdImpl(3818, "A space-separated list of the SSL ciphers to be used to communicate with distributed system.");
 -  public static final StringId AgentConfigImpl_DOES_CONNECTING_TO_THE_DISTRIBUTED_SYSTEM_REQUIRE_SSL_AUTHENTICATION = new StringIdImpl(3819, "Whether connection to the distributed system needs SSL authentication.");
 -  public static final StringId AgentConfigImpl_PROPERTY_FILE_FROM_WHICH_AGENT_READS_CONFIGURATION = new StringIdImpl(3820, "Name and path of the Agent''s properties file from which agent reads configuration");
 -  public static final StringId AgentConfigImpl_HOST_ON_WHICH_THE_DISTRIBUTED_SYSTEMS_LOCATOR_RUNS = new StringIdImpl(3821, "Host on which the distributed system''s locator runs");
 -  public static final StringId SystemAdmin_USED_TO_SPECIFY_THE_0_FILE_TO_BE_USED_IN_CONFIGURING_THE_LOCATORS_DISTRIBUTEDSYSTEM = new StringIdImpl(3822, "Used to specify the {0} file to be used in configuring the locator''s DistributedSystem.");
 -  public static final StringId AgentConfigImpl_GEMFIRE_PRODUCT_DIRECTORY_USED_TO_LAUNCH_A_LOCATOR = new StringIdImpl(3823, "GemFire product directory used to launch a locator");
 -  public static final StringId AgentConfigImpl_DIRECTORY_IN_WHICH_A_LOCATOR_WILL_BE_LAUNCHED = new StringIdImpl(3824, "Directory in which a locator will be launched");
 -  public static final StringId AgentConfigImpl_COMMAND_PREFIX_USED_WHEN_LAUNCHING_A_LOCATOR = new StringIdImpl(3825, "Command prefix used when launching a locator");
 -  public static final StringId AgentConfigImpl_IP_ADDRESS_TO_USE_WHEN_CONTACTING_LOCATOR = new StringIdImpl(3826, "IP address to use when contacting locator");
 -  public static final StringId AgentConfigImpl_PROPERTIES_FOR_CONFIGURING_A_LOCATORS_DISTRIBUTED_SYSTEM = new StringIdImpl(3827, "Properties for configuring a locator''s distributed system");
 -  public static final StringId AgentLauncher_STARTS_THE_GEMFIRE_JMX_AGENT = new StringIdImpl(3828, "Starts the GemFire JMX Agent");
 -  public static final StringId AgentLauncher_VMARG = new StringIdImpl(3829, "<vmarg> a VM-option passed to the agent''s VM, example -J-Xmx1024M for a 1 Gb heap");
 -  public static final StringId AgentLauncher_DIR = new StringIdImpl(3830, "<dir> Directory in which agent runs, default is the current directory");
 -  public static final StringId AgentLauncher_PROP = new StringIdImpl(3831, "<prop> A configuration property/value passed to the agent");
 -  public static final StringId AgentLauncher_SEE_HELP_CONFIG = new StringIdImpl(3832, "(see \"help config\" for more details)");
 -  public static final StringId AgentLauncher_STOPS_A_GEMFIRE_JMX_AGENT = new StringIdImpl(3833, "Stops a GemFire JMX Agent");
 -  public static final StringId AgentLauncher_REPORTS_THE_STATUS_AND_THE_PROCESS_ID_OF_A_GEMFIRE_JMX_AGENT = new StringIdImpl(3834, "Reports the status and the process id of a GemFire JMX Agent");
 -  public static final StringId AgentLauncher_AGENT_CONFIGURATION_PROPERTIES = new StringIdImpl(3835, "Agent configuration properties");
 -  public static final StringId AgentLauncher_DEFAULT = new StringIdImpl(3836, "Default");
 -  public static final StringId AgentLauncher_STARTING_JMX_AGENT_WITH_PID_0 = new StringIdImpl(3837, "Starting JMX Agent with pid: {0,number,#}");
 -  public static final StringId AgentLauncher_STARTING_AGENT = new StringIdImpl(3838, "Starting agent");
 -  public static final StringId AgentLauncher_UNCAUGHT_EXCEPTION_IN_THREAD_0 = new StringIdImpl(3839, "Uncaught exception in thread {0}");
 -  public static final StringId AgentLauncher_0_HAS_STOPPED = new StringIdImpl(3840, "The {0} has shut down.");
 -  public static final StringId AgentLauncher_TIMEOUT_WAITING_FOR_0_TO_SHUTDOWN_STATUS_IS_1 = new StringIdImpl(3841, "Timeout waiting for {0} to shutdown, status is: {1}");
 -  public static final StringId AgentLauncher_NO_HELP_AVAILABLE_FOR_0 = new StringIdImpl(3842, "No help available for \"{0}\"");
 -  public static final StringId AgentLauncher_AGENT_HELP = new StringIdImpl(3843, "agent help");
 -  public static final StringId AgentLauncher_UNKNOWN_COMMAND_0 = new StringIdImpl(3844, "Unknown command: {0}");
 -  public static final StringId AgentLauncher_ERROR_0 = new StringIdImpl(3845, "Error : {0}");
 -  public static final StringId AgentLauncher_0_PID_1_STATUS = new StringIdImpl(3846, "{0} pid: {1,number,#} status: ");
 -  public static final StringId AgentLauncher_SHUTDOWN = new StringIdImpl(3847, "shutdown");
 -  public static final StringId AgentLauncher_STARTING = new StringIdImpl(3848, "starting");
 -  public static final StringId AgentLauncher_RUNNING = new StringIdImpl(3849, "running");
 -  public static final StringId AgentLauncher_SHUTDOWN_PENDING = new StringIdImpl(3850, "shutdown pending");
 -  public static final StringId AgentLauncher_UNKNOWN = new StringIdImpl(3851, "unknown");
 -  public static final StringId MsgStreamer_CLOSING_DUE_TO_0 = new StringIdImpl(3852, "closing due to {0}");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_BEGIN_NESTED_TRANSACTION_IS_NOT_SUPPORTED = new StringIdImpl(3853, "Nested transaction is not supported");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_COMMIT_TRANSACTION_IS_NULL_CANNOT_COMMIT_A_NULL_TRANSACTION = new StringIdImpl(3854, "Transaction is null, cannot commit a null transaction");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_COMMIT_GLOBAL_TRANSACTION_IS_NULL_CANNOT_COMMIT_A_NULL_GLOBAL_TRANSACTION = new StringIdImpl(3855, "Global Transaction is null, cannot commit a null global transaction");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_COMMIT_TRANSACTION_NOT_ACTIVE_CANNOT_BE_COMMITTED_TRANSACTION_STATUS_0 = new StringIdImpl(3856, "transaction not active, cannot be committed. Transaction Status= {0}");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_COMMIT_TRANSACTION_IS_NOT_ACTIVE_AND_CANNOT_BE_COMMITTED = new StringIdImpl(3857, "transaction is not active and cannot be committed");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_COMMIT_TRANSACTION_ROLLED_BACK_BECAUSE_OF_EXCEPTION_IN_NOTIFYBEFORECOMPLETION_FUNCTION_CALL_ACTUAL_EXCEPTION_0 = new StringIdImpl(3858, "Transaction rolled back because of Exception in notifyBeforeCompletion processing");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_COMMIT_TRANSACTION_ROLLED_BACK_BECAUSE_A_USER_MARKED_IT_FOR_ROLLBACK = new StringIdImpl(3859, "Transaction rolled back because a user marked it for Rollback");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_ROLLBACK_NO_TRANSACTION_EXISTS = new StringIdImpl(3860, "no transaction exists");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_ROLLBACK_NO_GLOBAL_TRANSACTION_EXISTS = new StringIdImpl(3861, "no global transaction exists");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_ROLLBACK_TRANSACTION_STATUS_DOES_NOT_ALLOW_ROLLBACK_TRANSACTIONAL_STATUS_0 = new StringIdImpl(3862, "Transaction status does not allow Rollback .Transactional status = {0}");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_ROLLBACK_TRANSACTION_ALREADY_IN_A_ROLLING_BACK_STATE_TRANSACTIONAL_STATUS_0 = new StringIdImpl(3863, "Transaction already in a Rolling Back state.Transactional status = {0}");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_ROLLBACK_TRANSACTION_STATUS_DOES_NOT_ALLOW_ROLLBACK = new StringIdImpl(3864, "Transaction status does not allow Rollback");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_SETROLLBACKONLY_NO_GLOBAL_TRANSACTION_EXISTS = new StringIdImpl(3865, "no global transaction exists");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_SETROLLBACKONLY_TRANSACTION_CANNOT_BE_MARKED_FOR_ROLLBACK_TRANSCATION_STATUS_0 = new StringIdImpl(3866, "Transaction cannot be marked for rollback. Transcation status = {0}");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_SETTRANSACTIONTIMEOUT_NO_GLOBAL_TRANSACTION_EXISTS = new StringIdImpl(3867, "no global transaction exists");
 -  public static final StringId GemFireCacheImpl_STARTING_GEMFIRE_MEMCACHED_SERVER_ON_PORT_0_FOR_1_PROTOCOL = new StringIdImpl(3868, "Starting GemFireMemcachedServer on port {0} for {1} protocol");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_SETTRANSACTIONTIMEOUT_TRANSACTION_HAS_EITHER_EXPIRED_OR_ROLLEDBACK_OR_COMITTED = new StringIdImpl(3869, "Transaction has either expired or rolledback or comitted");
 -  public static final StringId TransactionManagerImpl_TRANSACTIONMANAGERIMPL_RESUME_CANNOT_RESUME_A_NULL_TRANSACTION = new StringIdImpl(3870, "cannot resume a null transaction");
 -  public static final StringId AbstractDistributionConfig_SECURITY_CLIENT_DHALGO_NAME_0 = new StringIdImpl(3871, "User defined name for the symmetric encryption algorithm to use in Diffie-Hellman key exchange for encryption of credentials.  Defaults to \"{0}\". Legal values can be any of the available symmetric algorithm names in JDK like \"DES\", \"DESede\", \"AES\", \"Blowfish\". It may be required to install Unlimited Strength Jurisdiction Policy Files from Sun for some symmetric algorithms to work (like \"AES\")");
 -  public

<TRUNCATED>


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

Posted by bs...@apache.org.
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);
 



[30/38] incubator-geode git commit: GEODE-385: fix test race

Posted by bs...@apache.org.
GEODE-385: fix test race

Region expiration was being enabled before the test suspended expiration.
This introduced a race in which the region could be expire destroyed
before the first put happened. Now expiry is suspended before region
expiration is configured.


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

Branch: refs/heads/feature/GEODE-77
Commit: 6c09b04caa55ad298c6cabcc591858f39c5aaaf8
Parents: 48b0c96
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Oct 6 11:24:48 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Oct 6 11:34:23 2015 -0700

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/TXExpiryJUnitTest.java | 15 ++++++++-------
 1 file changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6c09b04c/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
index 2a0a581..867ebb2 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/TXExpiryJUnitTest.java
@@ -318,14 +318,15 @@ public class TXExpiryJUnitTest {
       }
     };
     mutator.addCacheListener(cl);
-    if (useTTL) {
-      mutator.setRegionTimeToLive(exprAtt);
-    } else {
-      mutator.setRegionIdleTimeout(exprAtt);
-    }
-
+    // Suspend before enabling region expiration to prevent
+    // it from happening before we do the put.
+    ExpiryTask.suspendExpiration();
     try {
-      ExpiryTask.suspendExpiration();
+      if (useTTL) {
+        mutator.setRegionTimeToLive(exprAtt);
+      } else {
+        mutator.setRegionIdleTimeout(exprAtt);
+      }
 
       // Create some keys and age them, I wish we could fake/force the age
       // instead of having to actually wait


[10/38] incubator-geode git commit: fixed typo: giga -> tera

Posted by bs...@apache.org.
fixed typo: giga -> tera


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

Branch: refs/heads/feature/GEODE-77
Commit: 7111906e9fa21990ea91e3a940616fd003e05844
Parents: 1673e24
Author: Swapnil Bawaskar <sb...@pivotal.io>
Authored: Mon Oct 5 17:42:46 2015 -0700
Committer: Swapnil Bawaskar <sb...@pivotal.io>
Committed: Tue Oct 6 10:31:14 2015 -0700

----------------------------------------------------------------------
 README.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7111906e/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 8ac89a0..7c939bc 100755
--- a/README.md
+++ b/README.md
@@ -4,7 +4,7 @@ Geode is a data management platform that provides real-time, consistent access t
 
 Geode pools memory, CPU, network resources, and optionally local disk across multiple processes to manage application objects and behavior. It uses dynamic replication and data partitioning techniques to implement high availability, improved performance, scalability, and fault tolerance. In addition to being a distributed data container, Geode is an in-memory data management system that provides reliable asynchronous event notifications and guaranteed message delivery.
 
-Geode is an extremely mature and robust product that can trace its legacy all the way back to one of the first Object Databases for Smalltalk: GemStone. Geode (as GemFireâ„¢) was first deployed in the financial sector as the transactional, low-latency data engine used by multiple Wall Street trading platforms.  Today Geode is used by over 600 enterprise customers for high-scale, 24x7 business critical applications. An example deployment includes [China National Railways](http://pivotal.io/big-data/case-study/scaling-online-sales-for-the-largest-railway-in-the-world-china-railway-corporation) that uses Geode to run railway ticketing for the entire country of China with a 10 node cluster that manages 2 gigabytes of "hot data" in memory, and 10 backup nodes for high availability and elastic scale.
+Geode is an extremely mature and robust product that can trace its legacy all the way back to one of the first Object Databases for Smalltalk: GemStone. Geode (as GemFireâ„¢) was first deployed in the financial sector as the transactional, low-latency data engine used by multiple Wall Street trading platforms.  Today Geode is used by over 600 enterprise customers for high-scale, 24x7 business critical applications. An example deployment includes [China National Railways](http://pivotal.io/big-data/case-study/scaling-online-sales-for-the-largest-railway-in-the-world-china-railway-corporation) that uses Geode to run railway ticketing for the entire country of China with a 10 node cluster that manages 2 terabytes of "hot data" in memory, and 10 backup nodes for high availability and elastic scale.
 
 # Main Concepts and Components
 


[02/38] incubator-geode git commit: GEODE-379: fix spelling of entry-time-to-live-expiration-action option

Posted by bs...@apache.org.
GEODE-379: fix spelling of entry-time-to-live-expiration-action option


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

Branch: refs/heads/feature/GEODE-77
Commit: 35d3f06a9c4cf6f6491fd9d5ea5566e4682aa38c
Parents: d8570c9
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Oct 1 14:41:31 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Oct 1 14:41:31 2015 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/management/internal/cli/i18n/CliStrings.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/35d3f06a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
index 678fd93..a4561bf 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
@@ -566,7 +566,7 @@ public class CliStrings {
   public static final String CREATE_REGION__ENTRYEXPIRATIONIDLETIMEACTION__HELP = "Action to be taken on an entry that has exceeded the idle expiration.";
   public static final String CREATE_REGION__ENTRYEXPIRATIONTIMETOLIVE = "entry-time-to-live-expiration";
   public static final String CREATE_REGION__ENTRYEXPIRATIONTIMETOLIVE__HELP = "How long the region's entries can remain in the cache without being accessed or updated. The default is no expiration of this type.";
-  public static final String CREATE_REGION__ENTRYEXPIRATIONTTLACTION = "entry-time-to-live-expiriation-action";
+  public static final String CREATE_REGION__ENTRYEXPIRATIONTTLACTION = "entry-time-to-live-expiration-action";
   public static final String CREATE_REGION__ENTRYEXPIRATIONTTLACTION__HELP = "Action to be taken on an entry that has exceeded the TTL expiration.";
   public static final String CREATE_REGION__REGIONEXPIRATIONIDLETIME = "region-idle-time-expiration";
   public static final String CREATE_REGION__REGIONEXPIRATIONIDLETIME__HELP = "How long the region can remain in the cache without being accessed. The default is no expiration of this type.";


[04/38] incubator-geode git commit: [GEODE-157] Check for colocation not failing if non colocated comparison is first comparison made Fixed issue where if first region being checked happens to not be colocated, no longer ignores and throws the correct ex

Posted by bs...@apache.org.
[GEODE-157] Check for colocation not failing if non colocated comparison is first comparison made
Fixed issue where if first region being checked happens to not be colocated, no longer ignores and
throws the correct exception


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

Branch: refs/heads/feature/GEODE-77
Commit: 4eed17baa585146e4d9b5d686cbfdb84296b2233
Parents: 9e4a94e
Author: Jason Huynh <jh...@pivotal.io>
Authored: Mon Oct 5 09:47:43 2015 -0700
Committer: Jason Huynh <jh...@pivotal.io>
Committed: Mon Oct 5 09:47:43 2015 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/cache/query/internal/DefaultQuery.java | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4eed17ba/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java
index 3c87d7e..86a02a8 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/DefaultQuery.java
@@ -628,7 +628,7 @@ public class DefaultQuery implements Query {
       }
 
       // If there are more than one  PRs they have to be co-located.
-      QueryExecutor root = null;
+      QueryExecutor other = null;
       for (QueryExecutor eachPR : prs) {
         boolean colocated = false;
         
@@ -636,7 +636,7 @@ public class DefaultQuery implements Query {
           if (eachPR == allPRs) {
             continue;
           }
-
+          other = allPRs;
           if ((((PartitionedRegion) eachPR).colocatedByList.contains(allPRs) || 
               ((PartitionedRegion) allPRs).colocatedByList.contains(eachPR)))  {
             colocated = true;
@@ -644,13 +644,12 @@ public class DefaultQuery implements Query {
           } 
         } // allPrs
 
-        if (!colocated && root != null) {
+        if (!colocated) { 
           throw new UnsupportedOperationException(
               LocalizedStrings.DefaultQuery_A_QUERY_ON_A_PARTITIONED_REGION_0_MAY_NOT_REFERENCE_ANY_OTHER_NON_COLOCATED_PARTITIONED_REGION_1
               .toLocalizedString(new Object[] { eachPR.getName(),
-                  root.getName() }));
+                  other.getName() }));
         }
-        root = eachPR;
         
       } // eachPR
 


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeServerMaxConnectionsJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeServerMaxConnectionsJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeServerMaxConnectionsJUnitTest.java
deleted file mode 100644
index 60c9fba..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeServerMaxConnectionsJUnitTest.java
+++ /dev/null
@@ -1,221 +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.tier.sockets;
-
-import java.util.Properties;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.*;
-
-import junit.framework.TestCase;
-
-import com.gemstone.gemfire.Statistics;
-import com.gemstone.gemfire.StatisticsType;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.client.*;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.client.internal.Connection;
-import com.gemstone.gemfire.cache.util.BridgeServer;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-import dunit.DistributedTestCase;
-import dunit.DistributedTestCase.WaitCriterion;
-
-/**
- * Make sure max-connections on bridge server is enforced
- *
- * @author darrel
- *
- */
-@Category(IntegrationTest.class)
-public class BridgeServerMaxConnectionsJUnitTest
-
-{
-
-  /** connection proxy object for the client */
-  PoolImpl proxy = null;
-
-  /** the distributed system instance for the test */
-  DistributedSystem system;
-
-  /** the cache instance for the test */
-  Cache cache;
-
-  /** name of the region created */
-  final String regionName = "region1";
-
-  private static int PORT;
-
-  /**
-   * Close the cache and disconnects from the distributed system
-   *
-   * @exception -
-   *              thrown if any exception occured in closing cache/ds
-   */
-  @After
-  public void tearDown() throws Exception
-
-  {
-    this.cache.close();
-    this.system.disconnect();
-  }
-
-  /**
-   * Default to 0; override in sub tests to add thread pool
-   */
-  protected int getMaxThreads() {
-    return 0;
-  }
-
-  /**
-   * Initializes proxy object and creates region for client
-   *
-   */
-  private void createProxyAndRegionForClient()
-  {
-    try {
-      //props.setProperty("retryAttempts", "0");
-      PoolFactory pf = PoolManager.createFactory();
-      pf.addServer("localhost", PORT);
-      pf.setMinConnections(0);
-      pf.setPingInterval(10000);
-      pf.setThreadLocalConnections(true);
-      pf.setReadTimeout(2000);
-      pf.setSocketBufferSize(32768);
-      proxy = (PoolImpl)pf.create("junitPool");
-      AttributesFactory factory = new AttributesFactory();
-      factory.setScope(Scope.DISTRIBUTED_ACK);
-      factory.setPoolName("junitPool");
-      RegionAttributes attrs = factory.createRegionAttributes();
-      cache.createVMRegion(regionName, attrs);
-    }
-    catch (Exception ex) {
-      ex.printStackTrace();
-      fail("Failed to initialize client");
-    }
-  }
-
-  private final static int MAX_CNXS = 100;
-
-  /**
-   * Creates and starts the server instance
-   *
-   */
-  private int createServer()
-  {
-    BridgeServer server = null;
-    try {
-      Properties p = new Properties();
-      // make it a loner
-      p.put("mcast-port", "0");
-      p.put("locators", "");
-      this.system = DistributedSystem.connect(p);
-      this.cache = CacheFactory.create(system);
-      server = this.cache.addBridgeServer();
-      int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-      server.setMaxConnections(MAX_CNXS);
-      server.setMaxThreads(getMaxThreads());
-      server.setPort(port);
-      server.start();
-    }
-    catch (Exception e) {
-      e.printStackTrace();
-      fail("Failed to create server");
-    }
-    return server.getPort();
-  }
-
-  /**
-   * This test performs the following:<br>
-   * 1)create server<br>
-   * 2)initialize proxy object and create region for client<br>
-   * 3)perform a PUT on client by acquiring Connection through proxy<br>
-   * 4)stop server monitor threads in client to ensure that server treats this
-   * as dead client <br>
-   * 5)wait for some time to allow server to clean up the dead client artifacts<br>
-   * 6)again perform a PUT on client through same Connection and verify after
-   * the put that the Connection object used was new one.
-   */
-  @Test
-  public void testMaxCnxLimit() throws Exception
-  {
-    PORT = createServer();
-    createProxyAndRegionForClient();
-    StatisticsType st = this.system.findType("CacheServerStats");
-    final Statistics s = this.system.findStatisticsByType(st)[0];
-    assertEquals(0, s.getInt("currentClients"));
-    assertEquals(0, s.getInt("currentClientConnections"));
-    Connection[] cnxs = new Connection[MAX_CNXS];
-    for (int i=0; i < MAX_CNXS; i++) {
-      cnxs[i] = proxy.acquireConnection();
-      this.system.getLogWriter().info("acquired connection[" + i + "]=" + cnxs[i]);
-    }
-    WaitCriterion ev = new WaitCriterion() {
-      public boolean done() {
-        return s.getInt("currentClientConnections") == MAX_CNXS;
-      }
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 1000, 200, true);
-    assertEquals(MAX_CNXS, s.getInt("currentClientConnections"));
-    assertEquals(1, s.getInt("currentClients"));
-    this.system.getLogWriter().info("<ExpectedException action=add>" 
-        + "exceeded max-connections" + "</ExpectedException>");
-    try {
-      Connection cnx = proxy.acquireConnection();
-      if (cnx != null) {
-        fail("should not have been able to connect more than " + MAX_CNXS + " times but was able to connect " + s.getInt("currentClientConnections") + " times. Last connection=" + cnx);
-      }
-      this.system.getLogWriter().info("acquire connection returned null which is ok");
-    }
-    catch (NoAvailableServersException expected) {
-      // This is expected but due to race conditions in server handshake
-      // we may get null back from acquireConnection instead.
-      this.system.getLogWriter().info("received expected " + expected.getMessage());
-    }
-    catch (Exception ex) {
-      fail("expected acquireConnection to throw NoAvailableServersException but instead it threw " + ex);
-    }
-    finally {
-      this.system.getLogWriter().info("<ExpectedException action=remove>" 
-          + "exceeded max-connections" + "</ExpectedException>");
-    }
-
-    // now lets see what happens we we close our connections
-    for (int i=0; i < MAX_CNXS; i++) {
-      cnxs[i].close(false);
-    }
-    ev = new WaitCriterion() {
-      public boolean done() {
-        return s.getInt("currentClients") == 0;
-      }
-      public String description() {
-        return null;
-      }
-    };
-    DistributedTestCase.waitForCriterion(ev, 3 * 1000, 200, true);
-    this.system.getLogWriter().info("currentClients="
-        + s.getInt("currentClients")
-        + " currentClientConnections="
-        + s.getInt("currentClientConnections"));
-    assertEquals(0, s.getInt("currentClientConnections"));
-    assertEquals(0, s.getInt("currentClients"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeServerSelectorMaxConnectionsJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeServerSelectorMaxConnectionsJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeServerSelectorMaxConnectionsJUnitTest.java
deleted file mode 100644
index abe40c2..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeServerSelectorMaxConnectionsJUnitTest.java
+++ /dev/null
@@ -1,19 +0,0 @@
-package com.gemstone.gemfire.internal.cache.tier.sockets;
-
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-/**
- * Just like its parent but enables the server thread pool (aka selector).
- * 
- * @author darrel
- * 
- */
-@Category(IntegrationTest.class)
-public class BridgeServerSelectorMaxConnectionsJUnitTest extends BridgeServerMaxConnectionsJUnitTest
-{
-  protected int getMaxThreads() {
-    return 2; 
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeWriterMiscDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeWriterMiscDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeWriterMiscDUnitTest.java
deleted file mode 100755
index cbf732f..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeWriterMiscDUnitTest.java
+++ /dev/null
@@ -1,1384 +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.tier.sockets;
-
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheException;
-import com.gemstone.gemfire.cache.CacheWriterException;
-import com.gemstone.gemfire.cache.DataPolicy;
-import com.gemstone.gemfire.cache.EvictionAttributes;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.Scope;
-import com.gemstone.gemfire.cache.client.NoAvailableServersException;
-import com.gemstone.gemfire.cache.client.Pool;
-import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.cache.client.internal.Connection;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.client.internal.RegisterInterestTracker;
-import com.gemstone.gemfire.cache.util.BridgeServer;
-import com.gemstone.gemfire.cache.util.BridgeWriterException;
-import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
-import com.gemstone.gemfire.cache30.CacheTestCase;
-import com.gemstone.gemfire.cache30.LRUEvictionControllerDUnitTest;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeServerImpl;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-
-import dunit.DistributedTestCase;
-import dunit.Host;
-import dunit.SerializableCallable;
-import dunit.SerializableRunnable;
-import dunit.VM;
-
-import java.util.Iterator;
-import java.util.Properties;
-import java.util.Set;
-
-import junit.framework.AssertionFailedError;
-
-/**
- * Tests corner cases between Region, BridgeWriter and PoolImpl
- *
- * @author Yogesh Mahajan
- *
- */
-public class BridgeWriterMiscDUnitTest extends CacheTestCase
-{
-//  private static Cache cache = null;
-
-  protected static PoolImpl pool = null;
-
-  protected static Connection conn = null;
-  
-  private static Cache static_cache;
-
-  private static int PORT1;
-
-  private static final String k1 = "k1";
-
-  private static final String k2 = "k2";
-
-  private static final String server_k1 = "server-k1";
-
-  private static final String server_k2 = "server-k2";
-
-  private static final String REGION_NAME1 = "BridgeWriterMiscDUnitTest_region1";
-
-  private static final String REGION_NAME2 = "BridgeWriterMiscDUnitTest_region2";
-
-  private static final String PR_REGION_NAME = "BridgeWriterMiscDUnitTest_PRregion";
-
-  private static Host host;
-
-  private static VM server1;
-
-  private static VM server2;
-
-  private static RegionAttributes attrs;
-
-
-  // variables for concurrent map API test
-  Properties props = new Properties();
-  final int putRange_1Start = 1;
-  final int putRange_1End = 5;
-  final int putRange_2Start = 6;
-  final int putRange_2End = 10;
-  final int putRange_3Start = 11;
-  final int putRange_3End = 15;
-  final int putRange_4Start = 16;
-  final int putRange_4End = 20;
-  final int removeRange_1Start = 2;
-  final int removeRange_1End = 4;
-  final int removeRange_2Start = 7;
-  final int removeRange_2End = 9;
-
-  
-  
-  /** constructor */
-  public BridgeWriterMiscDUnitTest(String name) {
-    super(name);
-  }
-
-  public void setUp() throws Exception
-  {
-    super.setUp();
-    host = Host.getHost(0);
-    server1 = host.getVM(0);
-    server2 = host.getVM(1);
-  }
-
-  private int initServerCache(boolean notifyBySub) {
-    Object[] args = new Object[] { notifyBySub, getMaxThreads()};
-    return ((Integer)server1.invoke(BridgeWriterMiscDUnitTest.class,
-                                    "createServerCache",
-                                    args)).intValue();
-  }
-  
-  private int initServerCache2(boolean notifyBySub) {
-    Object[] args = new Object[] {notifyBySub, getMaxThreads()};
-    return ((Integer)server2.invoke(BridgeWriterMiscDUnitTest.class,
-                                    "createServerCache",
-                                    args)).intValue();
-  }
-
-  
-  public void testConcurrentOperationsWithDRandPR() throws Exception {
-    int port1 = initServerCache(true); // vm0
-    int port2 = initServerCache2(true); // vm1
-    String serverName = getServerHostName(Host.getHost(0));
-    host.getVM(2).invoke(this.getClass(), "createClientCacheV", new Object[]{serverName, port1});
-    host.getVM(3).invoke(this.getClass(), "createClientCacheV", new Object[]{serverName, port2});
-    getLogWriter().info("Testing concurrent map operations from a client with a distributed region");
-    concurrentMapTest(host.getVM(2), "/" + REGION_NAME1);
-    // TODO add verification in vm3
-    getLogWriter().info("Testing concurrent map operations from a client with a partitioned region");
-    concurrentMapTest(host.getVM(2), "/" + PR_REGION_NAME);
-    // TODO add verification in vm3
-  }
-
-  public void testConcurrentOperationsWithDRandPRandEmptyClient() throws Exception {
-    int port1 = initServerCache(true); // vm0
-    int port2 = initServerCache2(true); // vm1
-    String serverName = getServerHostName(Host.getHost(0));
-    host.getVM(2).invoke(this.getClass(), "createEmptyClientCache", new Object[]{serverName, port1});
-    host.getVM(3).invoke(this.getClass(), "createClientCacheV", new Object[]{serverName, port2});
-    getLogWriter().info("Testing concurrent map operations from a client with a distributed region");
-    concurrentMapTest(host.getVM(2), "/" + REGION_NAME1);
-    // TODO add verification in vm3
-    getLogWriter().info("Testing concurrent map operations from a client with a partitioned region");
-    concurrentMapTest(host.getVM(2), "/" + PR_REGION_NAME);
-    // TODO add verification in vm3
-  }
-
-  /**
-   * Do putIfAbsent(), replace(Object, Object),
-   * replace(Object, Object, Object), remove(Object, Object) operations
-   */
-  public void concurrentMapTest(final VM clientVM, final String rName) {
-    
-    //String exceptionStr = "";
-    clientVM.invoke(new CacheSerializableRunnable("doConcurrentMapOperations") {
-      public void run2() throws CacheException {
-        Cache cache = getCache();
-        final Region pr = cache.getRegion(rName);
-        assertNotNull(rName + " not created", pr);
-        boolean isEmpty = pr.getAttributes().getDataPolicy() == DataPolicy.EMPTY;
-        
-        // test successful putIfAbsent
-        for (int i = putRange_1Start; i <= putRange_1End; i++) {
-          Object putResult = pr.putIfAbsent(Integer.toString(i),
-                                            Integer.toString(i));
-          assertNull("Expected null, but got " + putResult + " for key " + i,
-                     putResult);
-        }
-        int size;
-        if (!isEmpty) {
-          size = pr.size();
-          assertEquals("Size doesn't return expected value", putRange_1End, size);
-          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
-              pr.isEmpty());
-        }
-        
-        // test unsuccessful putIfAbsent
-        for (int i = putRange_1Start; i <= putRange_1End; i++) {
-          Object putResult = pr.putIfAbsent(Integer.toString(i),
-                                            Integer.toString(i + 1));
-          assertEquals("for i=" + i, Integer.toString(i), putResult);
-          assertEquals("for i=" + i, Integer.toString(i), pr.get(Integer.toString(i)));
-        }
-        if (!isEmpty) {
-          size = pr.size();
-          assertEquals("Size doesn't return expected value", putRange_1End, size);
-          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
-              pr.isEmpty());
-        }
-               
-        // test successful replace(key, oldValue, newValue)
-        for (int i = putRange_1Start; i <= putRange_1End; i++) {
-         boolean replaceSucceeded = pr.replace(Integer.toString(i),
-                                               Integer.toString(i),
-                                               "replaced" + i);
-          assertTrue("for i=" + i, replaceSucceeded);
-          assertEquals("for i=" + i, "replaced" + i, pr.get(Integer.toString(i)));
-        }
-        if (!isEmpty) {
-          size = pr.size();
-          assertEquals("Size doesn't return expected value", putRange_1End, size);
-          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
-              pr.isEmpty());
-        }
-        
-        // test unsuccessful replace(key, oldValue, newValue)
-        for (int i = putRange_1Start; i <= putRange_2End; i++) {
-         boolean replaceSucceeded = pr.replace(Integer.toString(i),
-                                               Integer.toString(i), // wrong expected old value
-                                               "not" + i);
-         assertFalse("for i=" + i, replaceSucceeded);
-         assertEquals("for i=" + i,
-                      i <= putRange_1End ? "replaced" + i : null,
-                      pr.get(Integer.toString(i)));
-        }
-        if (!isEmpty) {
-          size = pr.size();
-          assertEquals("Size doesn't return expected value", putRange_1End, size);
-          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
-              pr.isEmpty());
-        }
-                                    
-        // test successful replace(key, value)
-        for (int i = putRange_1Start; i <= putRange_1End; i++) {
-          Object replaceResult = pr.replace(Integer.toString(i),
-                                            "twice replaced" + i);
-          assertEquals("for i=" + i, "replaced" + i, replaceResult);
-          assertEquals("for i=" + i,
-                       "twice replaced" + i,
-                       pr.get(Integer.toString(i)));
-        }
-        if (!isEmpty) {
-          size = pr.size();
-          assertEquals("Size doesn't return expected value", putRange_1End, size);
-          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
-              pr.isEmpty());
-        }
-                                    
-        // test unsuccessful replace(key, value)
-        for (int i = putRange_2Start; i <= putRange_2End; i++) {
-          Object replaceResult = pr.replace(Integer.toString(i),
-                                           "thrice replaced" + i);
-          assertNull("for i=" + i, replaceResult);
-          assertNull("for i=" + i, pr.get(Integer.toString(i)));
-        }
-        if (!isEmpty) {
-          size = pr.size();
-          assertEquals("Size doesn't return expected value", putRange_1End, size);
-          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
-              pr.isEmpty());
-        }
-                                    
-        // test unsuccessful remove(key, value)
-        for (int i = putRange_1Start; i <= putRange_2End; i++) {
-          boolean removeResult = pr.remove(Integer.toString(i),
-                                           Integer.toString(-i));
-          assertFalse("for i=" + i, removeResult);
-          assertEquals("for i=" + i,
-                       i <= putRange_1End ? "twice replaced" + i : null,
-                       pr.get(Integer.toString(i)));
-        }
-        if (!isEmpty) {
-          size = pr.size();
-          assertEquals("Size doesn't return expected value", putRange_1End, size);
-          assertFalse("isEmpty doesnt return proper state of the PartitionedRegion", 
-              pr.isEmpty());
-        }
-
-        // test successful remove(key, value)
-        for (int i = putRange_1Start; i <= putRange_1End; i++) {
-          boolean removeResult = pr.remove(Integer.toString(i),
-                                           "twice replaced" + i);
-          assertTrue("for i=" + i, removeResult);
-          assertEquals("for i=" + i, null, pr.get(Integer.toString(i)));
-        }
-        if (!isEmpty) {
-          size = pr.size();
-          assertEquals("Size doesn't return expected value", 0, size);
-          pr.localClear();
-          assertTrue("isEmpty doesnt return proper state of the PartitionedRegion", 
-              pr.isEmpty());
-        }
-        
-        if (!isEmpty) {
-          // bug #42169 - entry not updated on server when locally destroyed on client
-          String key42169 = "key42169";
-          pr.put(key42169, "initialValue42169");
-          pr.localDestroy(key42169);
-          boolean success = pr.replace(key42169, "initialValue42169", "newValue42169");
-          assertTrue("expected replace to succeed", success);
-          pr.destroy(key42169);
-          pr.put(key42169, "secondRound");
-          pr.localDestroy(key42169);
-          Object result = pr.putIfAbsent(key42169, null);
-          assertEquals("expected putIfAbsent to fail", result, "secondRound");
-          pr.destroy(key42169);
-        }
-        
-        if (isEmpty) {
-          String key41265 = "key41265";
-          boolean success = pr.remove(key41265, null);
-          assertFalse("expected remove to fail because key does not exist", success);
-        }
-        
-        // test null values
-        
-        // putIfAbsent with null value creates invalid entry
-        Object oldValue = pr.putIfAbsent("keyForNull", null);
-        assertNull(oldValue);
-        if (!isEmpty) {
-          assertTrue(pr.containsKey("keyForNull"));
-          assertTrue(!pr.containsValueForKey("keyForNull"));
-        }
-        
-        // replace allows null value for oldValue, meaning invalidated entry
-        assertTrue(pr.replace("keyForNull", null, "no longer invalid"));
-        
-        // replace does not allow null value for new value
-        try {
-          pr.replace("keyForNull", "no longer invalid", null);
-          fail("expected a NullPointerException");
-        } catch (NullPointerException expected) {
-        }
-        
-        // other variant of replace does not allow null value for new value
-        try {
-          pr.replace("keyForNull", null);
-          fail ("expected a NullPointerException");
-        } catch (NullPointerException expected) {
-        }
-        
-        // replace with null oldvalue matches invalidated entry
-        pr.putIfAbsent("otherKeyForNull", null);
-        int puts = ((GemFireCacheImpl)pr.getCache()).getCachePerfStats().getPuts();
-        boolean success = pr.replace("otherKeyForNull", null, "no longer invalid");
-        assertTrue(success);
-        int newputs = ((GemFireCacheImpl)pr.getCache()).getCachePerfStats().getPuts();
-        assertTrue("stats not updated properly or replace malfunctioned", newputs == puts+1);
-
-      }
-    });
-  }
-
-  /**
-   * Test for single BridgeWriter used across two regions: notify by subscription is true.
-   * For region1 the interest list is empty , for region 2 the intetest list is all keys.
-   * If an update/create is made on region1 , the client should not receive any.
-   * If the create/update is on region2 , the client should receive the update.
-   */
-  public void testSameBridgeWriterForTwoRegionHavingDifferentInterestList()
-      throws Exception
-  {
-    // start server first
-    PORT1 = initServerCache(true);
-    createClientCache(getServerHostName(Host.getHost(0)), PORT1);
-    populateCache();
-    registerInterest();
-    server1.invoke(BridgeWriterMiscDUnitTest.class, "put");
-
-//    pause(5000 + 5000 + 10000);
-    /*final int maxWaitTime = Integer.getInteger(WAIT_PROPERTY, WAIT_DEFAULT).intValue();
-    try {
-      Thread.yield();
-       Thread.sleep(maxWaitTime);
-    }
-    catch (InterruptedException e) {
-      fail("interrupted");
-    }*/
-    verifyUpdates();
-
-  }
-
-  /**
-   * Test for single BridgeWriter used across two regions: notify by subscription is true.
-   * Both the regions have registered interest in all the keys.
-   * Now close region1 on the client.
-   * The region1 should get removed from the interest list on CCP at server.
-   * Any update on region1 on server should not get pushed to the client.
-   * Ensure that the message related is not added to the client's queue at all
-   * ( which is diferent from not receiving a callbak on the client).
-   * If an update on region2 is made on the server , then client should receive the calback
-   */
-  public void testSameBridgeWriterForTwoRegionHavingALLKEYSInterest()
-      throws Exception
-  {
-    // start server first
-    PORT1 = initServerCache(true);
-    createClientCache(getServerHostName(Host.getHost(0)), PORT1);
-    populateCache();
-    registerInterestInBothTheRegions();
-    closeRegion1();
-    pause(6000);
-    server1.invoke(BridgeWriterMiscDUnitTest.class,
-        "verifyInterestListOnServer");
-    server1.invoke(BridgeWriterMiscDUnitTest.class, "put");
-    //pause(5000);
-    verifyUpdatesOnRegion2();
-  }
-
-  /** Test for single BridgeWriter used across two regions: notify by subscription is true.
-   * Both the regions have registered interest in all the keys.
-   * Close both the regions. When the last region is closed ,
-   * it should close the ConnectionProxy on the client ,
-   * close all the server connection threads on the server &
-   * remove the CacheClientProxy from the CacheClient notifier
-   */
-  public void testRegionCloseWithSameBridgeWriter() throws Exception
-  {
-    // start server first
-    PORT1 = initServerCache(true);
-    pool = (PoolImpl)createClientCache(getServerHostName(Host.getHost(0)),PORT1);
-    populateCache();
-    registerInterestInBothTheRegions();
-    closeBothRegions();
-    //pause(5000);
-    assertEquals(false, pool.isDestroyed());
-    pool.destroy();
-    assertEquals(true, pool.isDestroyed());
-    server1.invoke(BridgeWriterMiscDUnitTest.class,
-        "verifyNoCacheClientProxyOnServer");
-
-  }
-
-  /**
-   * Test for single BridgeWriter used across two regions: notify by
-   * subscription is true. Both the regions have registered interest in all the
-   * keys. Destroy region1 on the client. It should reach the server , kill the
-   * region on the server , propagate it to the interested clients , but it
-   * should keep CacheClient Proxy alive. Destroy Region2 . It should reach
-   * server , close conenction proxy , destroy the region2 on the server ,
-   * remove the cache client proxy from the cache client notifier & propagate it
-   * to the clients. Then create third region and verify that no
-   * CacheClientProxy is created on server
-   */
-  public void testCCPDestroyOnLastDestroyRegion() throws Exception
-  {
-    PORT1 = initServerCache(true);
-    PoolImpl pool = (PoolImpl)createClientCache(getServerHostName(Host.getHost(0)),PORT1);
-    destroyRegion1();
-    // pause(5000);
-    server1.invoke(BridgeWriterMiscDUnitTest.class,
-        "verifyCacheClientProxyOnServer", new Object[] { new String(
-            REGION_NAME1) });
-    Connection conn = pool.acquireConnection();
-    assertNotNull(conn);
-    assertEquals(1, pool.getConnectedServerCount());
-    assertEquals(false, pool.isDestroyed());
-    destroyRegion2();
-    assertEquals(false, pool.isDestroyed());
-    destroyPRRegion();
-    assertEquals(false, pool.isDestroyed());
-    pool.destroy();
-    assertEquals(true, pool.isDestroyed());
-    // pause(5000);
-    server1.invoke(BridgeWriterMiscDUnitTest.class,
-        "verifyNoCacheClientProxyOnServer");
-    try {
-      getCache().createRegion(REGION_NAME2, attrs);
-      fail("expected IllegalStateException");
-    } catch (IllegalStateException expected) {
-    }
-  }
-
-  /**
-   * Test for single BridgeWriter used across two regions:If notify by
-   * subscription is false , both the regions should receive invalidates for the
-   * updates on server in their respective regions
-   *
-   */
-  public void testInvalidatesPropagateOnTwoRegionsHavingCommonBridgeWriter()
-      throws Exception
-  {
-    // start server first
-    PORT1 = initServerCache(false);
-    createClientCache(getServerHostName(Host.getHost(0)), PORT1);
-    registerInterestForInvalidatesInBothTheRegions();
-    populateCache();
-    server1.invoke(BridgeWriterMiscDUnitTest.class, "put");
-    //pause(5000);
-    verifyInvalidatesOnBothRegions();
-
-  }
-
-  /**
-   * Test for bug 43407, where LRU in the client caused an entry to be
-   * evicted with DESTROY(), then the client invalidated the entry and
-   * did a get().  After the get() the entry was not seen to be in the
-   * client's cache.  This turned out to be expected behavior, but we
-   * now have this test to guarantee that the product behaves as expected.
-   */
-  public void testBug43407()
-      throws Exception
-  {
-    // start server first
-    PORT1 = initServerCache(false);
-    createClientCache(getServerHostName(Host.getHost(0)), PORT1);
-    registerInterestForInvalidatesInBothTheRegions();
-    Region region = static_cache.getRegion(REGION_NAME1);
-    populateCache();
-    region.put("invalidationKey", "invalidationValue");
-    region.localDestroy("invalidationKey");
-    if (region.containsKey("invalidationKey")) {
-      fail("region still contains invalidationKey");
-    }
-    region.invalidate("invalidationKey");
-    if (region.containsKey("invalidationKey")) {
-      fail("this test expects the entry is not created on invalidate() if not there before the operation");
-    }
-    Object value = region.get("invalidationKey");
-    if (value != null) {
-      fail("this test expected a null response to get('invalidationKey')");
-    }
-    if (!region.containsKeyOnServer("invalidationKey")) {
-      fail("expected an entry on the server after invalidation");
-    }
-    // bug 43407 asserts that there should be an entry, but the product does not
-    // do this.  This verifies that the product does not behave as asserted in that bug
-    if (region.containsKey("invalidationKey")) {
-      fail("expected no entry after invalidation when entry was not in client but was on server");
-    }
-  }
-
-  /**
-   * Create cache, create bridge writer, notify-by-subscription=false,
-   * create a region and on client and on server.
-   * Do not attach Bridge writer to region ,
-   * populate some entries on region both on client and server.
-   * Update the entries on server the client.
-   * The client should not have entry invalidate.
-   *
-   * @throws Exception
-   */
-  public void testInvalidatesPropagateOnRegionHavingNoBridgeWriter()
-      throws Exception
-  {
-    // start server first
-    PORT1 = initServerCache(false);
-    Properties props = new Properties();
-    props.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    props.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    new BridgeWriterMiscDUnitTest("temp").createCache(props);
-    String host = getServerHostName(server1.getHost());
-    PoolImpl p = (PoolImpl)PoolManager.createFactory()
-      .addServer(host, PORT1)
-      .setSubscriptionEnabled(true)
-      .setThreadLocalConnections(true)
-      .setReadTimeout(1000)
-      .setSocketBufferSize(32768)
-      .setMinConnections(3)
-      .setSubscriptionRedundancy(-1)
-      .setPingInterval(2000)
-      // .setRetryAttempts(5)
-      // .setRetryInterval(2000)
-      .create("testInvalidatesPropagateOnRegionHavingNoBridgeWriter");
-
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    // factory.setPoolName(p.getName());
-
-    attrs = factory.create();
-    final Region region1 = getCache().createRegion(REGION_NAME1, attrs);
-    final Region region2 = getCache().createRegion(REGION_NAME2, attrs);
-    assertNotNull(region1);
-    assertNotNull(region2);
-    pool = p;
-    conn = pool.acquireConnection();
-    assertNotNull(conn);
-
-    populateCache();
-    server1.invoke(BridgeWriterMiscDUnitTest.class, "put");
-    // pause(5000);
-    WaitCriterion wc = new WaitCriterion() {
-      String excuse;
-      public boolean done() {
-        Object val = region1.getEntry(k1).getValue();
-        return k1.equals(val);
-      }
-      public String description() {
-        return excuse;
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-    
-    // assertEquals(region1.getEntry(k1).getValue(), k1);
-    wc = new WaitCriterion() {
-      String excuse;
-      public boolean done() {
-        Object val = region1.getEntry(k2).getValue();
-        return k2.equals(val);
-      }
-      public String description() {
-        return excuse;
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-    
-    wc = new WaitCriterion() {
-      String excuse;
-      public boolean done() {
-        Object val = region2.getEntry(k1).getValue();
-        return k1.equals(val);
-      }
-      public String description() {
-        return excuse;
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-    
-    // assertEquals(region1.getEntry(k2).getValue(), k2);
-    // assertEquals(region2.getEntry(k1).getValue(), k1);
-    wc = new WaitCriterion() {
-      String excuse;
-      public boolean done() {
-        Object val = region2.getEntry(k2).getValue();
-        return k2.equals(val);
-      }
-      public String description() {
-        return excuse;
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-    
-    // assertEquals(region2.getEntry(k2).getValue(), k2);
-  }
-
-  /**
-   * Create proxy before cache creation, create cache, create two regions,
-   * attach same bridge writer to both of the regions Region interests AL_KEYS
-   * on both the regions, notify-by-subscription=true . The CCP should have both
-   * the regions in interest list.
-   *
-   * @throws Exception
-   */
-
-  public void testProxyCreationBeforeCacheCreation() throws Exception
-  {
-    Properties props = new Properties();
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
-    DistributedSystem ds = getSystem(props);
-    assertNotNull(ds);
-    ds.disconnect();
-    ds = getSystem(props);
-    PORT1 = initServerCache(true);
-    String host = getServerHostName(server1.getHost());
-    Pool p = PoolManager.createFactory()
-      .addServer(host, PORT1)
-      .setSubscriptionEnabled(true)
-      .setSubscriptionRedundancy(-1)
-      // .setRetryAttempts(5)
-      .create("testProxyCreationBeforeCacheCreationPool");
-
-    Cache cache = getCache();
-    assertNotNull(cache);
-
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setPoolName(p.getName());
-
-    RegionAttributes myAttrs = factory.create();
-    Region region1 = cache.createRegion(REGION_NAME1, myAttrs);
-    Region region2 = cache.createRegion(REGION_NAME2, myAttrs);
-    assertNotNull(region1);
-    assertNotNull(region2);
-    //region1.registerInterest(CacheClientProxy.ALL_KEYS);
-    region2.registerInterest("ALL_KEYS");
-    pause(6000);
-    server1.invoke(BridgeWriterMiscDUnitTest.class,
-        "verifyInterestListOnServer");
-
-  }
-  /**
-   * 
-   * Cycling a DistributedSystem with an initialized BridgeWriter causes interest registration NPE
-   * 
-   * Test Scenario:
-   *  
-   * Create a DistributedSystem (DS1). 
-   * Create a BridgeWriter (BW), initialize (creates a proxy with DS1 memberid) 
-   * Disconnect DS1.  Create a DistributedSystem (DS2). 
-   * Create a Region with BW, it attempts to register interest using DS2 memberid, gets NPE.
-   *  
-   * @throws Exception
-   */
-  public void testBug35380() throws Exception
-  {
-    Properties props = new Properties();
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
-    DistributedSystem ds = getSystem(props);
-    assertNotNull(ds);
-    
-    PORT1 = initServerCache(true);
-    String host = getServerHostName(server1.getHost());
-    Pool p = PoolManager.createFactory()
-      .addServer(host, PORT1)
-      .setSubscriptionEnabled(true)
-      .setSubscriptionRedundancy(-1)
-      //.setRetryAttempts(5)
-      .create("testBug35380Pool");
-
-    Cache cache = getCache();
-
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setPoolName(p.getName());
-
-    RegionAttributes myAttrs = factory.create();
-    Region region1 = cache.createRegion(REGION_NAME1, myAttrs);
-    Region region2 = cache.createRegion(REGION_NAME2, myAttrs);
-    assertNotNull(region1);
-    assertNotNull(region2);
-    
-    region2.registerInterest("ALL_KEYS");
-    
-    ds.disconnect();
-    Properties prop = new Properties();
-    prop.setProperty("mcast-port", "0");
-    prop.setProperty("locators", "");
-    ds = getSystem(prop);
-    
-    cache = getCache();
-    assertNotNull(cache);
-
-    AttributesFactory factory1 = new AttributesFactory();
-    factory1.setScope(Scope.DISTRIBUTED_ACK);
-    //reuse writer from prev DS
-    factory1.setPoolName(p.getName());
-
-    RegionAttributes attrs1 = factory1.create();
-    try {
-      cache.createRegion(REGION_NAME1, attrs1);
-      fail("expected ShutdownException");
-    }
-    catch(IllegalStateException expected) {
-    }
-    catch (BridgeWriterException expected) {
-    }
-    catch (DistributedSystemDisconnectedException expected) {
-    } 
-  }
-  
-
-  private void createCache(Properties props) throws Exception {
-    createCacheV(props);
-  }
-  private Cache createCacheV(Properties props) throws Exception
-  {
-    DistributedSystem ds = getSystem(props);
-    assertNotNull(ds);
-    ds.disconnect();
-    ds = getSystem(props);
-    Cache cache = getCache();
-    assertNotNull(cache);
-    return cache;
-  }
-
-  public static void createClientCacheV(String h, int port)
-  throws Exception {
-    _createClientCache(h, port, false);
-  }
-
-  public static void createEmptyClientCache(String h, int port)
-  throws Exception {
-    _createClientCache(h, port, true);
-  }
-
-  public static Pool createClientCache(String h, int port)
-  throws Exception  {
-    return _createClientCache(h, port, false);
-  }
-  
-  public static Pool _createClientCache(String h, int port, boolean empty)
-  throws Exception  {
-    Properties props = new Properties();
-    props.setProperty("mcast-port", "0");
-    props.setProperty("locators", "");
-    Cache cache = new BridgeWriterMiscDUnitTest("temp").createCacheV(props);
-    BridgeWriterMiscDUnitTest.static_cache = cache;
-    PoolImpl p = (PoolImpl)PoolManager.createFactory()
-      .addServer(h, port)
-      .setSubscriptionEnabled(true)
-      .setThreadLocalConnections(true)
-      .setReadTimeout(1000)
-      .setSocketBufferSize(32768)
-      .setMinConnections(3)
-      .setSubscriptionRedundancy(-1)
-      .setPingInterval(2000)
-      // .setRetryAttempts(5)
-      // .setRetryInterval(2000)
-      .create("BridgeWriterMiscDUnitTestPool");
-
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    if (empty) {
-      factory.setDataPolicy(DataPolicy.EMPTY);
-    }
-    factory.setPoolName(p.getName());
-
-    attrs = factory.create();
-    Region region1 = cache.createRegion(REGION_NAME1, attrs);
-    Region region2 = cache.createRegion(REGION_NAME2, attrs);
-    Region prRegion = cache.createRegion(PR_REGION_NAME, attrs);
-    assertNotNull(region1);
-    assertNotNull(region2);
-    assertNotNull(prRegion);
-    pool = p;
-//    conn = pool.acquireConnection();
-//    assertNotNull(conn);
-    // TODO does this WaitCriterion actually help?
-    WaitCriterion wc = new WaitCriterion() {
-      String excuse;
-      public boolean done() {
-        try {
-          conn = pool.acquireConnection();
-          if (conn == null) {
-            excuse = "acquireConnection returned null?";
-            return false;
-          }
-          return true;
-        } catch (NoAvailableServersException e) {
-          excuse = "Cannot find a server: " + e;
-          return false;
-        }
-      }
-      public String description() {
-        return excuse;
-      }
-    };
-    DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-    
-    return p;
-  }
-
-  public static Integer createServerCache(Boolean notifyBySubscription, Integer maxThreads)
-  throws Exception {
-    Cache cache = new BridgeWriterMiscDUnitTest("temp").createCacheV(new Properties());
-    unsetSlowDispatcherFlag();
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.DISTRIBUTED_ACK);
-    factory.setEnableConflation(true);
-    factory.setDataPolicy(DataPolicy.REPLICATE);
-    RegionAttributes myAttrs = factory.create();
-    Region r1 = cache.createRegion(REGION_NAME1, myAttrs);
-    Region r2 = cache.createRegion(REGION_NAME2, myAttrs);
-    factory = new AttributesFactory();
-    factory.setDataPolicy(DataPolicy.PARTITION);
-    RegionAttributes prAttrs = factory.create();
-    Region pr = cache.createRegion(PR_REGION_NAME, prAttrs);
-    assertNotNull(r1);
-    assertNotNull(r2);
-    assertNotNull(pr);
-
-    BridgeServer server = cache.addBridgeServer();
-    int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    r1.getCache().getDistributedSystem().getLogWriter().info("Starting server on port " + port);
-    server.setPort(port);
-    server.setMaxThreads(maxThreads.intValue());
-    server.setNotifyBySubscription(notifyBySubscription.booleanValue());
-    server.start();
-    r1.getCache().getDistributedSystem().getLogWriter().info("Started server on port " + server.getPort());
-    return new Integer(server.getPort());
-
-  }
-
-  protected int getMaxThreads() {
-    return 0; 
-  }
-
-  public static void registerInterest()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      assertNotNull(r);
-      //r.registerInterestRegex(CacheClientProxy.ALL_KEYS);
-      r.registerInterest("ALL_KEYS");
-    }
-    catch (CacheWriterException e) {
-      e.printStackTrace();
-      fail("Test failed due to CacheWriterException during registerInterest", e);
-    }
-  }
-
-  public static void registerInterestForInvalidatesInBothTheRegions()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
-      assertNotNull(r1);
-      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      assertNotNull(r2);
-      r1.registerInterest("ALL_KEYS", false, false);
-      r2.registerInterest("ALL_KEYS", false, false);
-    }
-    catch (CacheWriterException e) {
-      e.printStackTrace();
-      fail(
-          "Test failed due to CacheWriterException during registerInterestnBothRegions",
-          e);
-    }
-  }
-
-  public static void registerInterestInBothTheRegions()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
-      assertNotNull(r1);
-      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      assertNotNull(r2);
-      r1.registerInterest("ALL_KEYS");
-      r2.registerInterest("ALL_KEYS");
-    }
-    catch (CacheWriterException e) {
-      e.printStackTrace();
-      fail(
-          "Test failed due to CacheWriterException during registerInterestnBothRegions",
-          e);
-    }
-  }
-
-  public static void closeRegion1()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
-      assertNotNull(r1);
-      r1.close();
-    }
-    catch (Exception e) {
-      e.printStackTrace();
-      fail("Test failed due to Exception during closeRegion1", e);
-    }
-  }
-
-  public static void closeBothRegions()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
-      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      Region pr = cache.getRegion(Region.SEPARATOR + PR_REGION_NAME);
-      assertNotNull(r1);
-      assertNotNull(r2);
-      assertNotNull(pr);
-      r1.close();
-      r2.close();
-      pr.close();
-    }
-    catch (Exception e) {
-      e.printStackTrace();
-      fail("Test failed due to Exception during closeBothRegions", e);
-    }
-  }
-
-  public static void destroyRegion1()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
-      assertNotNull(r1);
-      r1.destroyRegion();
-    }
-    catch (Exception e) {
-      e.printStackTrace();
-      fail("Test failed due to Exception during closeBothRegions", e);
-    }
-  }
-
-  public static void destroyRegion2()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      assertNotNull(r2);
-      r2.destroyRegion();
-    }
-    catch (Exception e) {
-      e.printStackTrace();
-      fail("Test failed due to Exception during closeBothRegions", e);
-    }
-  }
-
-  public static void destroyPRRegion()  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r2 = cache.getRegion(Region.SEPARATOR + PR_REGION_NAME);
-      assertNotNull(r2);
-      r2.destroyRegion();
-    } catch (Exception e) {
-     // e.printStackTrace();
-      fail("Test failed due to Exception during closeBothRegions", e);
-    }
-  }
-
-  public static void verifyInterestListOnServer()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      assertEquals("More than one BridgeServer", 1, cache.getBridgeServers()
-          .size());
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
-          .iterator().next();
-      assertNotNull(bs);
-      assertNotNull(bs.getAcceptor());
-      assertNotNull(bs.getAcceptor().getCacheClientNotifier());
-      Iterator iter_prox = bs.getAcceptor().getCacheClientNotifier()
-          .getClientProxies().iterator();
-      while (iter_prox.hasNext()) {
-        CacheClientProxy ccp = (CacheClientProxy)iter_prox.next();
-        // CCP should not contain region1
-        Set akr = ccp.cils[RegisterInterestTracker.interestListIndex].regions;
-        assertNotNull(akr);
-        assertTrue(!akr.contains(Region.SEPARATOR + REGION_NAME1));
-        // CCP should contain region2
-        assertTrue(akr.contains(Region.SEPARATOR + REGION_NAME2));
-        assertEquals(1, akr.size());
-      }
-    }
-    catch (Exception ex) {
-      ex.printStackTrace();
-      fail("while setting verifyInterestListOnServer  " + ex);
-    }
-  }
-
-  public static void verifyNoCacheClientProxyOnServer()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      assertEquals("More than one BridgeServer", 1, cache.getBridgeServers()
-          .size());
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
-          .iterator().next();
-      assertNotNull(bs);
-      assertNotNull(bs.getAcceptor());
-      final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-
-      assertNotNull(ccn);
-      WaitCriterion wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          return ccn.getClientProxies().size() == 0;
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 40 * 1000, 1000, true);
-    }
-    catch (Exception ex) {
-      ex.printStackTrace();
-      fail("while setting verifyNoCacheClientProxyOnServer  " + ex);
-    }
-  }
-
-  public static void verifyCacheClientProxyOnServer(String regionName)
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      assertNull(cache.getRegion(Region.SEPARATOR + regionName));
-       verifyCacheClientProxyOnServer();
-
-      //assertEquals(1, bs.getAcceptor().getCacheClientNotifier().getClientProxies().size());
-    }
-    catch (Exception ex) {
-      ex.printStackTrace();
-      fail("while setting verifyNoCacheClientProxyOnServer  " + ex);
-    }
-  }
-
-  public static void verifyCacheClientProxyOnServer()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      assertEquals("More than one BridgeServer", 1, cache.getBridgeServers()
-          .size());
-      BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers()
-          .iterator().next();
-      assertNotNull(bs);
-      assertNotNull(bs.getAcceptor());
-      final CacheClientNotifier ccn = bs.getAcceptor().getCacheClientNotifier();
-
-      assertNotNull(ccn);
-      WaitCriterion wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          return ccn.getClientProxies().size() == 1;
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 40 * 1000, 1000, true);
-    }
-    catch (Exception ex) {
-      ex.printStackTrace();
-      fail("while setting verifyNoCacheClientProxyOnServer  " + ex);
-    }
-  }
-
-  public static void populateCache()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
-      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      assertNotNull(r1);
-      assertNotNull(r2);
-
-      if (!r1.containsKey(k1))
-        r1.create(k1, k1);
-      if (!r1.containsKey(k2))
-        r1.create(k2, k2);
-      if (!r2.containsKey(k1))
-        r2.create(k1, k1);
-      if (!r2.containsKey(k2))
-        r2.create(k2, k2);
-
-      assertEquals(r1.getEntry(k1).getValue(), k1);
-      assertEquals(r1.getEntry(k2).getValue(), k2);
-      assertEquals(r2.getEntry(k1).getValue(), k1);
-      assertEquals(r2.getEntry(k2).getValue(), k2);
-    }
-    catch (Exception ex) {
-      fail("failed while createEntries()", ex);
-    }
-  }
-
-  public static void put()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
-      Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      assertNotNull(r1);
-      assertNotNull(r2);
-
-      r1.put(k1, server_k1);
-      r1.put(k2, server_k2);
-
-      r2.put(k1, server_k1);
-      r2.put(k2, server_k2);
-
-      assertEquals(r1.getEntry(k1).getValue(), server_k1);
-      assertEquals(r1.getEntry(k2).getValue(), server_k2);
-      assertEquals(r2.getEntry(k1).getValue(), server_k1);
-      assertEquals(r2.getEntry(k2).getValue(), server_k2);
-    }
-    catch (Exception ex) {
-      fail("failed while put()", ex);
-    }
-  }
-
-  public static void verifyUpdates()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
-      final Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      assertNotNull(r1);
-      assertNotNull(r2);
-      // verify updates
-      WaitCriterion wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r1.getEntry(k1).getValue();
-          return k1.equals(val);
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-      
-      // assertEquals(k1, r1.getEntry(k1).getValue());
-      wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r1.getEntry(k2).getValue();
-          return k2.equals(val);
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-      
-      // assertEquals(k2, r1.getEntry(k2).getValue());
-      wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r2.getEntry(k1).getValue();
-          return server_k1.equals(val);
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-      
-      // assertEquals(server_k1, r2.getEntry(k1).getValue());
-      wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r2.getEntry(k2).getValue();
-          return server_k2.equals(val);
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-      
-      // assertEquals(server_k2, r2.getEntry(k2).getValue());
-    }
-    catch (Exception ex) {
-      fail("failed while verifyUpdates()", ex);
-    }
-  }
-
-  public static void verifyInvalidatesOnBothRegions()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      final Region r1 = cache.getRegion(Region.SEPARATOR + REGION_NAME1);
-      final Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      assertNotNull(r1);
-      assertNotNull(r2);
-      
-      WaitCriterion wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r1.getEntry(k1).getValue();
-          return val == null;
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
-      
-      // assertNull(r1.getEntry(k1).getValue());
-      wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r1.getEntry(k1).getValue();
-          return val == null;
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
-      
-      // assertNull(r1.getEntry(k2).getValue());
-      wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r1.getEntry(k2).getValue();
-          return val == null;
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
-      
-
-      // assertNull(r2.getEntry(k1).getValue());
-      wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r2.getEntry(k1).getValue();
-          return val == null;
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
-      
-      // assertNull(r2.getEntry(k2).getValue());
-      wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r2.getEntry(k2).getValue();
-          return val == null;
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 90 * 1000, 1000, true);
-    }
-    catch (Exception ex) {
-      fail("failed while verifyInvalidatesOnBothRegions()", ex);
-    }
-  }
-
-  public static void verifyUpdatesOnRegion2()
-  {
-    try {
-      Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-      final Region r2 = cache.getRegion(Region.SEPARATOR + REGION_NAME2);
-      assertNotNull(r2);
-      WaitCriterion wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r2.getEntry(k1).getValue();
-          return server_k1.equals(val);
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-      
-      // assertEquals(server_k1, r2.getEntry(k1).getValue());
-      wc = new WaitCriterion() {
-        String excuse;
-        public boolean done() {
-          Object val = r2.getEntry(k2).getValue();
-          return server_k2.equals(val);
-        }
-        public String description() {
-          return excuse;
-        }
-      };
-      DistributedTestCase.waitForCriterion(wc, 60 * 1000, 1000, true);
-      
-      // assertEquals(server_k2, r2.getEntry(k2).getValue());
-    }
-    catch (Exception ex) {
-      fail("failed while verifyUpdatesOnRegion2()", ex);
-    }
-  }
-
-  public void tearDown2() throws Exception
-  {
-    super.tearDown2();
-    // close the clients first
-    closeCache();
-    // then close the servers
-    server1.invoke(BridgeWriterMiscDUnitTest.class, "closeCache");
-
-  }
-
-  public static void closeCache()
-  {
-    Cache cache = new BridgeWriterMiscDUnitTest("temp").getCache();
-    if (cache != null && !cache.isClosed()) {
-      cache.close();
-      cache.getDistributedSystem().disconnect();
-    }
-  }
-
-  /**
-   * set the boolean for starting the dispatcher thread a bit later to FALSE.
-   * This is just a precaution in case any test set it to true and did not unset
-   * it on completion.
-   *
-   */
-  public static void unsetSlowDispatcherFlag()
-  {
-    CacheClientProxy.isSlowStartForTesting = false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeWriterMiscSelectorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeWriterMiscSelectorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeWriterMiscSelectorDUnitTest.java
deleted file mode 100644
index 01bc903..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/BridgeWriterMiscSelectorDUnitTest.java
+++ /dev/null
@@ -1,27 +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.tier.sockets;
-
-
-/**
- * Just like BridgeWriterMiscDUnitTest but enables bridge thread pool
- * (ie. selector)
- *
- * @author darrel
- *
- */
-public class BridgeWriterMiscSelectorDUnitTest extends BridgeWriterMiscDUnitTest
-{
-  public BridgeWriterMiscSelectorDUnitTest(String name) {
-    super(name);
-  }
-
-  protected int getMaxThreads() {
-    return 2; 
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java
index 2a159bf..0c20400 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36269DUnitTest.java
@@ -19,7 +19,7 @@ import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.internal.Connection;
 import com.gemstone.gemfire.cache.client.internal.PoolImpl;
 import com.gemstone.gemfire.cache.client.internal.ServerRegionProxy;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
@@ -160,7 +160,7 @@ public class Bug36269DUnitTest extends DistributedTestCase
     factory.setScope(Scope.DISTRIBUTED_ACK);
     factory.setMirrorType(MirrorType.KEYS_VALUES);
     cache.createRegion(REGION_NAME, factory.create());
-    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/tier/sockets/Bug36457DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36457DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36457DUnitTest.java
index 5ffd2b7..f1a149e 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36457DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36457DUnitTest.java
@@ -21,13 +21,13 @@ import com.gemstone.gemfire.cache.Scope;
 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.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.BridgeObserverAdapter;
-import com.gemstone.gemfire.internal.cache.BridgeObserverHolder;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverAdapter;
+import com.gemstone.gemfire.internal.cache.ClientServerObserverHolder;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
@@ -121,7 +121,7 @@ public class Bug36457DUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
     cache.createRegion(regionName, attrs);
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
     server1.start();
@@ -159,7 +159,7 @@ public class Bug36457DUnitTest extends DistributedTestCase
         getServerHostName(server1.getHost()), port1, port2 });
     //set a cllabck so that we come to know that whether a failover is called or not
     // if failover is called means this bug is present.
-    client2.invoke(Bug36457DUnitTest.class, "setBridgeObserver");
+    client2.invoke(Bug36457DUnitTest.class, "setClientServerObserver");
     client1.invoke(Bug36457DUnitTest.class, "destroyRegion");
     isFaileoverHappened = ((Boolean)client2.invoke(Bug36457DUnitTest.class,
         "isFaileoverHappened")).booleanValue();
@@ -173,11 +173,11 @@ public class Bug36457DUnitTest extends DistributedTestCase
     return new Boolean(isFaileoverHappened);
   }
 
-  public static void setBridgeObserver()
+  public static void setClientServerObserver()
   {
     PoolImpl.AFTER_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = true;
-    BridgeObserverHolder
-        .setInstance(new BridgeObserverAdapter() {
+    ClientServerObserverHolder
+        .setInstance(new ClientServerObserverAdapter() {
           public void afterPrimaryIdentificationFromBackup(ServerLocation primaryEndpoint)
           {
             getLogWriter().fine("TEST FAILED HERE YOGI ");
@@ -186,7 +186,7 @@ public class Bug36457DUnitTest extends DistributedTestCase
         });
   }
 
-  public static void unSetBridgeObserver()
+  public static void unSetClientServerObserver()
   {
     PoolImpl.AFTER_PRIMARY_IDENTIFICATION_FROM_BACKUP_CALLBACK_FLAG = false;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36805DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36805DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36805DUnitTest.java
index 88cd873..d149094 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36805DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36805DUnitTest.java
@@ -21,7 +21,7 @@ import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.ServerOperationException;
 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.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -115,7 +115,7 @@ public class Bug36805DUnitTest extends DistributedTestCase
     new Bug36805DUnitTest("temp").createCache(new Properties());
    // no region is created on server 
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.setNotifyBySubscription(true);
     server1.start();
@@ -154,7 +154,7 @@ public class Bug36805DUnitTest extends DistributedTestCase
     // set a cllabck so that we come to know that whether a failover is called
     // or not
     // if failover is called means this bug is present.
-    // client2.invoke(Bug36805DUnitTest.class, "setBridgeObserver");
+    // client2.invoke(Bug36805DUnitTest.class, "setClientServerObserver");
     client1.invoke(Bug36805DUnitTest.class, "registerInterest"); // register
                                                                   // interest
                                                                   // shoud not

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36995DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36995DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36995DUnitTest.java
index 0ede87a..e858dce 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36995DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36995DUnitTest.java
@@ -14,7 +14,7 @@ import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.util.BridgeServer;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
@@ -121,7 +121,7 @@ public class Bug36995DUnitTest extends DistributedTestCase
     new Bug36995DUnitTest("temp").createCache(new Properties());
     // no region is created on server 
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
-    BridgeServer server1 = cache.addBridgeServer();
+    CacheServer server1 = cache.addCacheServer();
     server1.setPort(port);
     server1.start();
     return new Integer(server1.getPort());
@@ -180,9 +180,6 @@ public class Bug36995DUnitTest extends DistributedTestCase
 
   /**
    * BugTest for 36526 : 
-   * getActiveServers() and getDeadServers() on BridgeLoader and BridgeWriter returns a Set of endpoint names.  
-   * Endpoints are stored internally as a list and allow duplicate logical names.  
-   * If any duplicates, getActiveServers() will omit them.
    */
   public void testBug36526()
   {
@@ -202,9 +199,9 @@ public class Bug36995DUnitTest extends DistributedTestCase
   public static void stopServer()
   {
     try {
-      Iterator iter = cache.getBridgeServers().iterator();
+      Iterator iter = cache.getCacheServers().iterator();
       if (iter.hasNext()) {
-        BridgeServer server = (BridgeServer)iter.next();
+        CacheServer server = (CacheServer)iter.next();
         server.stop();
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37210DUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37210DUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37210DUnitTest.java
index 6800456..fd83e2b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37210DUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37210DUnitTest.java
@@ -18,11 +18,11 @@ 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.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 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.ha.HARegionQueue;
 import com.gemstone.gemfire.cache.client.*;
 
@@ -166,7 +166,7 @@ public class Bug37210DUnitTest extends DistributedTestCase
     RegionAttributes attrs = factory.create();
 
     cache.createRegion(REGION_NAME, attrs);
-    BridgeServer server = cache.addBridgeServer();
+    CacheServer server = cache.addCacheServer();
     int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     server.setPort(port);
     server.setNotifyBySubscription(false);
@@ -223,8 +223,8 @@ public class Bug37210DUnitTest extends DistributedTestCase
   public static void closeCacheClientProxyAndVerifyStats()
   {
     assertEquals("More than one BridgeServers found ", 1, cache
-        .getBridgeServers().size());
-    BridgeServerImpl bs = (BridgeServerImpl)cache.getBridgeServers().iterator()
+        .getCacheServers().size());
+    CacheServerImpl bs = (CacheServerImpl)cache.getCacheServers().iterator()
         .next();
     assertNotNull(bs);
     assertNotNull(bs.getAcceptor());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java
new file mode 100644
index 0000000..ba21c39
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerMaxConnectionsJUnitTest.java
@@ -0,0 +1,220 @@
+/*=========================================================================
+ * 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.tier.sockets;
+
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.*;
+import junit.framework.TestCase;
+
+import com.gemstone.gemfire.Statistics;
+import com.gemstone.gemfire.StatisticsType;
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.client.*;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.cache.client.internal.Connection;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.internal.AvailablePort;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import dunit.DistributedTestCase;
+import dunit.DistributedTestCase.WaitCriterion;
+
+/**
+ * Make sure max-connections on cache server is enforced
+ *
+ * @author darrel
+ *
+ */
+@Category(IntegrationTest.class)
+public class CacheServerMaxConnectionsJUnitTest
+
+{
+
+  /** connection proxy object for the client */
+  PoolImpl proxy = null;
+
+  /** the distributed system instance for the test */
+  DistributedSystem system;
+
+  /** the cache instance for the test */
+  Cache cache;
+
+  /** name of the region created */
+  final String regionName = "region1";
+
+  private static int PORT;
+
+  /**
+   * Close the cache and disconnects from the distributed system
+   *
+   * @exception -
+   *              thrown if any exception occured in closing cache/ds
+   */
+  @After
+  public void tearDown() throws Exception
+
+  {
+    this.cache.close();
+    this.system.disconnect();
+  }
+
+  /**
+   * Default to 0; override in sub tests to add thread pool
+   */
+  protected int getMaxThreads() {
+    return 0;
+  }
+
+  /**
+   * Initializes proxy object and creates region for client
+   *
+   */
+  private void createProxyAndRegionForClient()
+  {
+    try {
+      //props.setProperty("retryAttempts", "0");
+      PoolFactory pf = PoolManager.createFactory();
+      pf.addServer("localhost", PORT);
+      pf.setMinConnections(0);
+      pf.setPingInterval(10000);
+      pf.setThreadLocalConnections(true);
+      pf.setReadTimeout(2000);
+      pf.setSocketBufferSize(32768);
+      proxy = (PoolImpl)pf.create("junitPool");
+      AttributesFactory factory = new AttributesFactory();
+      factory.setScope(Scope.DISTRIBUTED_ACK);
+      factory.setPoolName("junitPool");
+      RegionAttributes attrs = factory.createRegionAttributes();
+      cache.createVMRegion(regionName, attrs);
+    }
+    catch (Exception ex) {
+      ex.printStackTrace();
+      fail("Failed to initialize client");
+    }
+  }
+
+  private final static int MAX_CNXS = 100;
+
+  /**
+   * Creates and starts the server instance
+   *
+   */
+  private int createServer()
+  {
+    CacheServer server = null;
+    try {
+      Properties p = new Properties();
+      // make it a loner
+      p.put("mcast-port", "0");
+      p.put("locators", "");
+      this.system = DistributedSystem.connect(p);
+      this.cache = CacheFactory.create(system);
+      server = this.cache.addCacheServer();
+      int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
+      server.setMaxConnections(MAX_CNXS);
+      server.setMaxThreads(getMaxThreads());
+      server.setPort(port);
+      server.start();
+    }
+    catch (Exception e) {
+      e.printStackTrace();
+      fail("Failed to create server");
+    }
+    return server.getPort();
+  }
+
+  /**
+   * This test performs the following:<br>
+   * 1)create server<br>
+   * 2)initialize proxy object and create region for client<br>
+   * 3)perform a PUT on client by acquiring Connection through proxy<br>
+   * 4)stop server monitor threads in client to ensure that server treats this
+   * as dead client <br>
+   * 5)wait for some time to allow server to clean up the dead client artifacts<br>
+   * 6)again perform a PUT on client through same Connection and verify after
+   * the put that the Connection object used was new one.
+   */
+  @Test
+  public void testMaxCnxLimit() throws Exception
+  {
+    PORT = createServer();
+    createProxyAndRegionForClient();
+    StatisticsType st = this.system.findType("CacheServerStats");
+    final Statistics s = this.system.findStatisticsByType(st)[0];
+    assertEquals(0, s.getInt("currentClients"));
+    assertEquals(0, s.getInt("currentClientConnections"));
+    Connection[] cnxs = new Connection[MAX_CNXS];
+    for (int i=0; i < MAX_CNXS; i++) {
+      cnxs[i] = proxy.acquireConnection();
+      this.system.getLogWriter().info("acquired connection[" + i + "]=" + cnxs[i]);
+    }
+    WaitCriterion ev = new WaitCriterion() {
+      public boolean done() {
+        return s.getInt("currentClientConnections") == MAX_CNXS;
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 1000, 200, true);
+    assertEquals(MAX_CNXS, s.getInt("currentClientConnections"));
+    assertEquals(1, s.getInt("currentClients"));
+    this.system.getLogWriter().info("<ExpectedException action=add>" 
+        + "exceeded max-connections" + "</ExpectedException>");
+    try {
+      Connection cnx = proxy.acquireConnection();
+      if (cnx != null) {
+        fail("should not have been able to connect more than " + MAX_CNXS + " times but was able to connect " + s.getInt("currentClientConnections") + " times. Last connection=" + cnx);
+      }
+      this.system.getLogWriter().info("acquire connection returned null which is ok");
+    }
+    catch (NoAvailableServersException expected) {
+      // This is expected but due to race conditions in server handshake
+      // we may get null back from acquireConnection instead.
+      this.system.getLogWriter().info("received expected " + expected.getMessage());
+    }
+    catch (Exception ex) {
+      fail("expected acquireConnection to throw NoAvailableServersException but instead it threw " + ex);
+    }
+    finally {
+      this.system.getLogWriter().info("<ExpectedException action=remove>" 
+          + "exceeded max-connections" + "</ExpectedException>");
+    }
+
+    // now lets see what happens we we close our connections
+    for (int i=0; i < MAX_CNXS; i++) {
+      cnxs[i].close(false);
+    }
+    ev = new WaitCriterion() {
+      public boolean done() {
+        return s.getInt("currentClients") == 0;
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 3 * 1000, 200, true);
+    this.system.getLogWriter().info("currentClients="
+        + s.getInt("currentClients")
+        + " currentClientConnections="
+        + s.getInt("currentClientConnections"));
+    assertEquals(0, s.getInt("currentClientConnections"));
+    assertEquals(0, s.getInt("currentClients"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerSelectorMaxConnectionsJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerSelectorMaxConnectionsJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerSelectorMaxConnectionsJUnitTest.java
new file mode 100644
index 0000000..7c8f84b
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheServerSelectorMaxConnectionsJUnitTest.java
@@ -0,0 +1,19 @@
+package com.gemstone.gemfire.internal.cache.tier.sockets;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Just like its parent but enables the server thread pool (aka selector).
+ * 
+ * @author darrel
+ * 
+ */
+@Category(IntegrationTest.class)
+public class CacheServerSelectorMaxConnectionsJUnitTest extends CacheServerMaxConnectionsJUnitTest
+{
+  protected int getMaxThreads() {
+    return 2; 
+  }
+}


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGemFireVM.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGemFireVM.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGemFireVM.java
index 8c18e61..ab64026 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGemFireVM.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/admin/remote/RemoteGemFireVM.java
@@ -672,7 +672,7 @@ public abstract class RemoteGemFireVM implements GemFireVM {
     return setCacheConfigValue(c, SEARCH_TIMEOUT_CODE, v);
   }
 
-  public AdminBridgeServer addBridgeServer(CacheInfo cache) 
+  public AdminBridgeServer addCacheServer(CacheInfo cache) 
     throws AdminException {
 
     BridgeServerRequest request =

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractBridgeServer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractBridgeServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractBridgeServer.java
deleted file mode 100644
index 383239e..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractBridgeServer.java
+++ /dev/null
@@ -1,425 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-package com.gemstone.gemfire.internal.cache;
-
-import com.gemstone.gemfire.CancelException;
-import com.gemstone.gemfire.cache.Cache;
-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.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.util.BridgeServer;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.distributed.internal.DM;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.admin.ClientMembershipMessage;
-import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Set;
-
-/**
- * Abstract class that contains common code that all true implementations
- * of {@link CacheServer} can use.
- *
- * @author darrel
- * @since 5.7
- */
-public abstract class AbstractBridgeServer implements CacheServer, BridgeServer {
-
-  public static final String TEST_OVERRIDE_DEFAULT_PORT_PROPERTY = "gemfire.test.CacheServer.OVERRIDE_DEFAULT_PORT";
-
-  /** The cache that is served by this bridge server */
-  protected final InternalCache cache;
-
-  /** The port that the bridge server was configured to run on */
-  protected int port;
-  
-  /** The maximum number of connections that the BridgeServer will accept */
-  protected int maxConnections;
-
-  /** The maximum number of threads that the BridgeServer will create */
-  protected int maxThreads;
-
-  /** Whether the bridge server notifies by subscription */
-  protected boolean notifyBySubscription = true;
-  
-  /**
-   * The buffer size in bytes of the socket for this 
-   * <code>BridgeServer</code>
-   */
-  protected int socketBufferSize;
-  
-  /**
-   * The tcpNoDelay setting for outgoing sockets
-   */
-  protected boolean tcpNoDelay;
-  
-  /**
-   * The maximum amount of time between client pings. This value is used by
-   * the <code>ClientHealthMonitor</code> to determine the health of this
-   * <code>BridgeServer</code>'s clients.
-   */
-  protected int maximumTimeBetweenPings;
-  
-  /** the maximum number of messages that can be enqueued in a client-queue. */
-  protected int maximumMessageCount;
-  
-  /**
-   * the time (in seconds) after which a message in the client queue will
-   * expire.
-   */
-  protected int messageTimeToLive;
-  /**
-   * The groups this server belongs to. Use <code>getGroups</code> to read.
-   * @since 5.7
-   */
-  protected String[] groups;
-  
-  protected ServerLoadProbe loadProbe;
-
-  /**
-   * The ip address or host name that this server is to listen on.
-   * @since 5.7
-   */
-  protected String bindAddress;
-  /**
-   * The ip address or host name that will be given to clients so they can connect
-   * to this server
-   * @since 5.7
-   */
-  protected String hostnameForClients;
-  
-  /**
-   * How frequency to poll the load on this server.
-   */
-  protected long loadPollInterval;
-  
-  protected ClientSubscriptionConfig clientSubscriptionConfig;
-  
-  /**
-   * Listener that would listen to bridge membership and notify the admin 
-   * members(if any exist) as clients of this server leave/crash. 
-   */
-  protected final BridgeMembershipListener listener;
-
-  /**
-   * The number of seconds to keep transaction states for disconnected clients.
-   * This allows the client to fail over to another server and still find
-   * the transaction state to complete the transaction.
-   */
-  private int transactionTimeToLive;
-  
-  //////////////////////  Constructors  //////////////////////
-
-  /**
-   * Creates a new <code>BridgeServer</code> with the default
-   * configuration.
-   *
-   * @param cache
-   *        The cache being served
-   */
-  public AbstractBridgeServer(InternalCache cache) {
-    this(cache, true);
-  }
-
-  public AbstractBridgeServer(InternalCache cache, boolean attachListener) {
-    this.cache = cache;
-    this.port = Integer.getInteger(TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, CacheServer.DEFAULT_PORT);
-    this.maxConnections = CacheServer.DEFAULT_MAX_CONNECTIONS;
-    this.maxThreads = CacheServer.DEFAULT_MAX_THREADS;
-    this.socketBufferSize = CacheServer.DEFAULT_SOCKET_BUFFER_SIZE;
-    this.tcpNoDelay = CacheServer.DEFAULT_TCP_NO_DELAY;
-    this.maximumTimeBetweenPings = CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS;
-    this.maximumMessageCount = CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT;
-    this.messageTimeToLive = CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE;
-    // TODO this should be configurable in CacheServer
-    this.transactionTimeToLive = Integer.getInteger("gemfire.cacheServer.transactionTimeToLive", 180);
-    this.groups = CacheServer.DEFAULT_GROUPS;
-    this.bindAddress = CacheServer.DEFAULT_BIND_ADDRESS;
-    this.hostnameForClients = CacheServer.DEFAULT_HOSTNAME_FOR_CLIENTS;
-    this.loadProbe = CacheServer.DEFAULT_LOAD_PROBE;
-    this.loadPollInterval = CacheServer.DEFAULT_LOAD_POLL_INTERVAL;
-    this.clientSubscriptionConfig = new ClientSubscriptionConfigImpl();
-
-    if (!attachListener) {
-      this.listener = null;
-      return;
-    }
-    listener = new BridgeMembershipListenerAdapter() {
-      /**
-       * Invoked when a client connected to this process or when this process 
-       * has got connected with a BridgeServer.
-       * 
-       * @param event
-       *          BridgeMembershipEvent associated with client getting connected
-       */
-      @Override
-      public void memberJoined(BridgeMembershipEvent event) {
-        /* process events for clients only */
-        if (event.isClient()) {
-          createAndSendMessage(event, ClientMembershipMessage.JOINED);
-        }
-      }
-      
-      /**
-       * Invoked when a client has gracefully disconnected from this process
-       * or when this process has gracefully disconnected from a BridgeServer.
-       * 
-       * @param event
-       *          BridgeMembershipEvent associated with client leaving gracefully
-       */
-      @Override
-      public void memberLeft(BridgeMembershipEvent event) {
-        /* process events for clients only */
-        if (event.isClient()) {
-          createAndSendMessage(event, ClientMembershipMessage.LEFT);
-        }
-      }
-
-      /**
-       * Invoked when a client has unexpectedly disconnected from this process
-       * or when this process has unexpectedly disconnected from a BridgeServer.
-       * 
-       * @param event
-       *          BridgeMembershipEvent associated with client getting
-       *          disconnected unexpectedly
-       */
-      @Override
-      public void memberCrashed(BridgeMembershipEvent event) {
-        /* process events for clients only */
-        if (event.isClient()) {
-          createAndSendMessage(event, ClientMembershipMessage.CRASHED);
-        }
-      }
-
-      /**
-       * Method to create & send the ClientMembershipMessage to admin members.
-       * The message is sent only if there are any admin members in the
-       * distribution system.
-       * 
-       * @param event
-       *          BridgeMembershipEvent associated for a change in client
-       *          membership
-       * @param type
-       *          type of event - one of ClientMembershipMessage.JOINED,
-       *          ClientMembershipMessage.LEFT, ClientMembershipMessage.CRASHED
-       */
-      private void createAndSendMessage(BridgeMembershipEvent event, int type) {
-        InternalDistributedSystem ds = null;
-        Cache cacheInstance = AbstractBridgeServer.this.cache;
-        if (cacheInstance != null && !(cacheInstance instanceof CacheCreation)) {
-          ds = (InternalDistributedSystem)cacheInstance.getDistributedSystem();
-        } else {
-          ds = InternalDistributedSystem.getAnyInstance();
-        }
-
-        //ds could be null
-        if (ds != null && ds.isConnected()) {
-          DM dm =  ds.getDistributionManager();
-          Set adminMemberSet = dm.getAdminMemberSet();
-
-          /* check if there are any admin members at all */
-          if (!adminMemberSet.isEmpty()) {
-            DistributedMember member = event.getMember();
-
-            ClientMembershipMessage msg = 
-              new ClientMembershipMessage(event.getMemberId(), 
-                                      member == null ? null : member.getHost(), 
-                                      type);
-            
-            msg.setRecipients(adminMemberSet);
-            dm.putOutgoing(msg);
-          }
-        }
-      }
-    };
-
-    BridgeMembership.registerBridgeMembershipListener(listener);
-  }
-
-  /////////////////////  Instance Methods  /////////////////////
-
-  public int getPort() {
-    return this.port;
-  }
-
-  public void setPort(int port) {
-    this.port = port;
-  }
-
-  public String getBindAddress() {
-    return this.bindAddress;
-  }
-
-  public void setBindAddress(String address) {
-    this.bindAddress = address;
-  }
-  
-  public String getHostnameForClients() {
-    return this.hostnameForClients;
-  }
-
-  public void setHostnameForClients(String name) {
-    this.hostnameForClients = name;
-  }
-  
-  public int getMaxConnections() {
-    return this.maxConnections;
-  }
-
-  public void setMaxConnections(int maxCon) {
-    this.maxConnections = maxCon;
-  }
-
-  public int getMaxThreads() {
-    return this.maxThreads;
-  }
-
-  public void setMaxThreads(int maxThreads) {
-    this.maxThreads = maxThreads;
-  }
-
-  public void start() throws IOException {
-    // This method is invoked during testing, but it is not necessary
-    // to do anything.
-  }
-
-  public void setNotifyBySubscription(boolean b) {
-    //this.notifyBySubscription = true;
-  }
-
-  public boolean getNotifyBySubscription() {
-    return this.notifyBySubscription;
-  }
-
-  public void setSocketBufferSize(int socketBufferSize) {
-    this.socketBufferSize = socketBufferSize;
-  }
-  
-  public int getSocketBufferSize() {
-    return this.socketBufferSize;
-  }
-
-  public void setMaximumTimeBetweenPings(int maximumTimeBetweenPings) {
-    this.maximumTimeBetweenPings = maximumTimeBetweenPings;
-  }
-  
-  public int getMaximumTimeBetweenPings() {
-    return this.maximumTimeBetweenPings;
-  }
-  
-  public int getMaximumMessageCount() {
-    return this.maximumMessageCount;
-  }
-
-  public void setMaximumMessageCount(int maximumMessageCount) {
-    this.maximumMessageCount = maximumMessageCount;
-  }
-  
-  public void setTransactionTimeToLive(int seconds) {
-    this.transactionTimeToLive = seconds;
-  }
-  
-  public int getTransactionTimeToLive() {
-    return this.transactionTimeToLive;
-  }
-  
-  public int getMessageTimeToLive() {
-    return this.messageTimeToLive;
-  }
-
-  public void setMessageTimeToLive(int messageTimeToLive) {
-    this.messageTimeToLive = messageTimeToLive;
-  }
-  
-  public void setGroups(String[] groups) {
-    if (groups == null) {
-      this.groups = CacheServer.DEFAULT_GROUPS;
-    }
-    else if (groups.length > 0) {
-      // copy it for isolation
-      String [] copy = new String[groups.length];
-      System.arraycopy(groups, 0, copy, 0, groups.length);
-      this.groups = copy;
-    } else {
-      this.groups = CacheServer.DEFAULT_GROUPS; // keep findbugs happy
-    }
-  }
-
-  public String[] getGroups() {
-    String[] result = this.groups;
-    if (result.length > 0) {
-      // copy it for isolation
-      String [] copy = new String[result.length];
-      System.arraycopy(result, 0, copy, 0, result.length);
-      result = copy;
-    }
-    return result;
-  }
-  
-  public ServerLoadProbe getLoadProbe() {
-    return loadProbe;
-  }
-
-  public void setLoadProbe(ServerLoadProbe loadProbe) {
-    this.loadProbe = loadProbe;
-  }
-  
-  public long getLoadPollInterval() {
-    return loadPollInterval;
-  }
-
-  public void setLoadPollInterval(long loadPollInterval) {
-    this.loadPollInterval = loadPollInterval;
-  }
-
-  public void setTcpNoDelay(boolean setting) {
-    this.tcpNoDelay = setting;
-  }
-  
-  public boolean getTcpNoDelay() {
-    return this.tcpNoDelay;
-  }
-
-  public Cache getCache() {
-    return this.cache;
-  }
-
-  private static boolean eq(String s1, String s2) {
-    if (s1 == null) {
-      return s2 == null;
-    } else {
-      return s1.equals(s2);
-    }
-  }
-  
-  /**
-   * Returns whether or not this bridge server has the same
-   * configuration as another bridge server.
-   */
-  public boolean sameAs(CacheServer other) {
-    return getPort() == other.getPort()
-      && eq(getBindAddress(), other.getBindAddress())
-      && getSocketBufferSize() == other.getSocketBufferSize()
-      && getMaximumTimeBetweenPings() == other.getMaximumTimeBetweenPings()
-      && getNotifyBySubscription() == other.getNotifyBySubscription()
-      && getMaxConnections() == other.getMaxConnections()
-      && getMaxThreads() == other.getMaxThreads()
-      && getMaximumMessageCount() == other.getMaximumMessageCount()
-      && getMessageTimeToLive() == other.getMessageTimeToLive()
-      && Arrays.equals(getGroups(), other.getGroups())
-      && getLoadProbe().equals(other.getLoadProbe())
-      && getLoadPollInterval() == other.getLoadPollInterval()
-      && getTcpNoDelay() == other.getTcpNoDelay();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractCacheServer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractCacheServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractCacheServer.java
new file mode 100644
index 0000000..42b6bcd
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractCacheServer.java
@@ -0,0 +1,398 @@
+/*=========================================================================
+ * Copyright (c) 2002-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
+ * more patents listed at http://www.pivotal.io/patents.
+ *=========================================================================
+ */
+package com.gemstone.gemfire.internal.cache;
+
+import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.cache.Cache;
+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.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.admin.ClientMembershipMessage;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheCreation;
+import com.gemstone.gemfire.management.membership.ClientMembership;
+import com.gemstone.gemfire.management.membership.ClientMembershipEvent;
+import com.gemstone.gemfire.management.membership.ClientMembershipListener;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+
+/**
+ * Abstract class that contains common code that all true implementations
+ * of {@link CacheServer} can use.
+ *
+ * @author darrel
+ * @since 5.7
+ */
+public abstract class AbstractCacheServer implements CacheServer {
+
+  public static final String TEST_OVERRIDE_DEFAULT_PORT_PROPERTY = "gemfire.test.CacheServer.OVERRIDE_DEFAULT_PORT";
+
+  /** The cache that is served by this bridge server */
+  protected final InternalCache cache;
+
+  /** The port that the bridge server was configured to run on */
+  protected int port;
+  
+  /** The maximum number of connections that the BridgeServer will accept */
+  protected int maxConnections;
+
+  /** The maximum number of threads that the BridgeServer will create */
+  protected int maxThreads;
+
+  /** Whether the bridge server notifies by subscription */
+  protected boolean notifyBySubscription = true;
+  
+  /**
+   * The buffer size in bytes of the socket for this 
+   * <code>BridgeServer</code>
+   */
+  protected int socketBufferSize;
+  
+  /**
+   * The tcpNoDelay setting for outgoing sockets
+   */
+  protected boolean tcpNoDelay;
+  
+  /**
+   * The maximum amount of time between client pings. This value is used by
+   * the <code>ClientHealthMonitor</code> to determine the health of this
+   * <code>BridgeServer</code>'s clients.
+   */
+  protected int maximumTimeBetweenPings;
+  
+  /** the maximum number of messages that can be enqueued in a client-queue. */
+  protected int maximumMessageCount;
+  
+  /**
+   * the time (in seconds) after which a message in the client queue will
+   * expire.
+   */
+  protected int messageTimeToLive;
+  /**
+   * The groups this server belongs to. Use <code>getGroups</code> to read.
+   * @since 5.7
+   */
+  protected String[] groups;
+  
+  protected ServerLoadProbe loadProbe;
+
+  /**
+   * The ip address or host name that this server is to listen on.
+   * @since 5.7
+   */
+  protected String bindAddress;
+  /**
+   * The ip address or host name that will be given to clients so they can connect
+   * to this server
+   * @since 5.7
+   */
+  protected String hostnameForClients;
+  
+  /**
+   * How frequency to poll the load on this server.
+   */
+  protected long loadPollInterval;
+  
+  protected ClientSubscriptionConfig clientSubscriptionConfig;
+  
+  /**
+   * Listens to client membership events and notifies any admin 
+   * members as clients of this server leave/crash. 
+   */
+  protected final ClientMembershipListener listener;
+
+  /**
+   * The number of seconds to keep transaction states for disconnected clients.
+   * This allows the client to fail over to another server and still find
+   * the transaction state to complete the transaction.
+   */
+  private int transactionTimeToLive;
+  
+  //////////////////////  Constructors  //////////////////////
+
+  /**
+   * Creates a new <code>BridgeServer</code> with the default
+   * configuration.
+   *
+   * @param cache
+   *        The cache being served
+   */
+  public AbstractCacheServer(InternalCache cache) {
+    this(cache, true);
+  }
+
+  public AbstractCacheServer(InternalCache cache, boolean attachListener) {
+    this.cache = cache;
+    this.port = Integer.getInteger(TEST_OVERRIDE_DEFAULT_PORT_PROPERTY, CacheServer.DEFAULT_PORT);
+    this.maxConnections = CacheServer.DEFAULT_MAX_CONNECTIONS;
+    this.maxThreads = CacheServer.DEFAULT_MAX_THREADS;
+    this.socketBufferSize = CacheServer.DEFAULT_SOCKET_BUFFER_SIZE;
+    this.tcpNoDelay = CacheServer.DEFAULT_TCP_NO_DELAY;
+    this.maximumTimeBetweenPings = CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS;
+    this.maximumMessageCount = CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT;
+    this.messageTimeToLive = CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE;
+    // TODO this should be configurable in CacheServer
+    this.transactionTimeToLive = Integer.getInteger("gemfire.cacheServer.transactionTimeToLive", 180);
+    this.groups = CacheServer.DEFAULT_GROUPS;
+    this.bindAddress = CacheServer.DEFAULT_BIND_ADDRESS;
+    this.hostnameForClients = CacheServer.DEFAULT_HOSTNAME_FOR_CLIENTS;
+    this.loadProbe = CacheServer.DEFAULT_LOAD_PROBE;
+    this.loadPollInterval = CacheServer.DEFAULT_LOAD_POLL_INTERVAL;
+    this.clientSubscriptionConfig = new ClientSubscriptionConfigImpl();
+
+    if (!attachListener) {
+      this.listener = null;
+      return;
+    }
+    listener = new ClientMembershipListener() {
+
+      @Override
+      public void memberJoined(ClientMembershipEvent event) {
+        if (event.isClient()) {
+          createAndSendMessage(event, ClientMembershipMessage.JOINED);
+        }
+      }
+
+      @Override
+      public void memberLeft(ClientMembershipEvent event) {
+        if (event.isClient()) {
+          createAndSendMessage(event, ClientMembershipMessage.LEFT);
+        }
+      }
+
+      @Override
+      public void memberCrashed(ClientMembershipEvent event) {
+        if (event.isClient()) {
+          createAndSendMessage(event, ClientMembershipMessage.CRASHED);
+        }
+      }
+      
+      /**
+       * Method to create & send the ClientMembershipMessage to admin members.
+       * The message is sent only if there are any admin members in the
+       * distribution system.
+       * 
+       * @param event
+       *          describes a change in client membership
+       * @param type
+       *          type of event - one of ClientMembershipMessage.JOINED,
+       *          ClientMembershipMessage.LEFT, ClientMembershipMessage.CRASHED
+       */
+      private void createAndSendMessage(ClientMembershipEvent event, int type) {
+        InternalDistributedSystem ds = null;
+        Cache cacheInstance = AbstractCacheServer.this.cache;
+        if (cacheInstance != null && !(cacheInstance instanceof CacheCreation)) {
+          ds = (InternalDistributedSystem)cacheInstance.getDistributedSystem();
+        } else {
+          ds = InternalDistributedSystem.getAnyInstance();
+        }
+
+        //ds could be null
+        if (ds != null && ds.isConnected()) {
+          DM dm =  ds.getDistributionManager();
+          Set adminMemberSet = dm.getAdminMemberSet();
+
+          /* check if there are any admin members at all */
+          if (!adminMemberSet.isEmpty()) {
+            DistributedMember member = event.getMember();
+
+            ClientMembershipMessage msg = 
+              new ClientMembershipMessage(event.getMemberId(), 
+                                      member == null ? null : member.getHost(), 
+                                      type);
+            
+            msg.setRecipients(adminMemberSet);
+            dm.putOutgoing(msg);
+          }
+        }
+      }
+    };
+
+    ClientMembership.registerClientMembershipListener(listener);
+  }
+
+  /////////////////////  Instance Methods  /////////////////////
+
+  public int getPort() {
+    return this.port;
+  }
+
+  public void setPort(int port) {
+    this.port = port;
+  }
+
+  public String getBindAddress() {
+    return this.bindAddress;
+  }
+
+  public void setBindAddress(String address) {
+    this.bindAddress = address;
+  }
+  
+  public String getHostnameForClients() {
+    return this.hostnameForClients;
+  }
+
+  public void setHostnameForClients(String name) {
+    this.hostnameForClients = name;
+  }
+  
+  public int getMaxConnections() {
+    return this.maxConnections;
+  }
+
+  public void setMaxConnections(int maxCon) {
+    this.maxConnections = maxCon;
+  }
+
+  public int getMaxThreads() {
+    return this.maxThreads;
+  }
+
+  public void setMaxThreads(int maxThreads) {
+    this.maxThreads = maxThreads;
+  }
+
+  public void start() throws IOException {
+    // This method is invoked during testing, but it is not necessary
+    // to do anything.
+  }
+
+  public void setNotifyBySubscription(boolean b) {
+    //this.notifyBySubscription = true;
+  }
+
+  public boolean getNotifyBySubscription() {
+    return this.notifyBySubscription;
+  }
+
+  public void setSocketBufferSize(int socketBufferSize) {
+    this.socketBufferSize = socketBufferSize;
+  }
+  
+  public int getSocketBufferSize() {
+    return this.socketBufferSize;
+  }
+
+  public void setMaximumTimeBetweenPings(int maximumTimeBetweenPings) {
+    this.maximumTimeBetweenPings = maximumTimeBetweenPings;
+  }
+  
+  public int getMaximumTimeBetweenPings() {
+    return this.maximumTimeBetweenPings;
+  }
+  
+  public int getMaximumMessageCount() {
+    return this.maximumMessageCount;
+  }
+
+  public void setMaximumMessageCount(int maximumMessageCount) {
+    this.maximumMessageCount = maximumMessageCount;
+  }
+  
+  public void setTransactionTimeToLive(int seconds) {
+    this.transactionTimeToLive = seconds;
+  }
+  
+  public int getTransactionTimeToLive() {
+    return this.transactionTimeToLive;
+  }
+  
+  public int getMessageTimeToLive() {
+    return this.messageTimeToLive;
+  }
+
+  public void setMessageTimeToLive(int messageTimeToLive) {
+    this.messageTimeToLive = messageTimeToLive;
+  }
+  
+  public void setGroups(String[] groups) {
+    if (groups == null) {
+      this.groups = CacheServer.DEFAULT_GROUPS;
+    }
+    else if (groups.length > 0) {
+      // copy it for isolation
+      String [] copy = new String[groups.length];
+      System.arraycopy(groups, 0, copy, 0, groups.length);
+      this.groups = copy;
+    } else {
+      this.groups = CacheServer.DEFAULT_GROUPS; // keep findbugs happy
+    }
+  }
+
+  public String[] getGroups() {
+    String[] result = this.groups;
+    if (result.length > 0) {
+      // copy it for isolation
+      String [] copy = new String[result.length];
+      System.arraycopy(result, 0, copy, 0, result.length);
+      result = copy;
+    }
+    return result;
+  }
+  
+  public ServerLoadProbe getLoadProbe() {
+    return loadProbe;
+  }
+
+  public void setLoadProbe(ServerLoadProbe loadProbe) {
+    this.loadProbe = loadProbe;
+  }
+  
+  public long getLoadPollInterval() {
+    return loadPollInterval;
+  }
+
+  public void setLoadPollInterval(long loadPollInterval) {
+    this.loadPollInterval = loadPollInterval;
+  }
+
+  public void setTcpNoDelay(boolean setting) {
+    this.tcpNoDelay = setting;
+  }
+  
+  public boolean getTcpNoDelay() {
+    return this.tcpNoDelay;
+  }
+
+  public Cache getCache() {
+    return this.cache;
+  }
+
+  private static boolean eq(String s1, String s2) {
+    if (s1 == null) {
+      return s2 == null;
+    } else {
+      return s1.equals(s2);
+    }
+  }
+  
+  /**
+   * Returns whether or not this bridge server has the same
+   * configuration as another bridge server.
+   */
+  public boolean sameAs(CacheServer other) {
+    return getPort() == other.getPort()
+      && eq(getBindAddress(), other.getBindAddress())
+      && getSocketBufferSize() == other.getSocketBufferSize()
+      && getMaximumTimeBetweenPings() == other.getMaximumTimeBetweenPings()
+      && getNotifyBySubscription() == other.getNotifyBySubscription()
+      && getMaxConnections() == other.getMaxConnections()
+      && getMaxThreads() == other.getMaxThreads()
+      && getMaximumMessageCount() == other.getMaximumMessageCount()
+      && getMessageTimeToLive() == other.getMessageTimeToLive()
+      && Arrays.equals(getGroups(), other.getGroups())
+      && getLoadProbe().equals(other.getLoadProbe())
+      && getLoadPollInterval() == other.getLoadPollInterval()
+      && getTcpNoDelay() == other.getTcpNoDelay();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java
index 90dcb41..344155b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegion.java
@@ -74,9 +74,6 @@ import com.gemstone.gemfire.cache.query.SelectResults;
 import com.gemstone.gemfire.cache.query.TypeMismatchException;
 import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
 import com.gemstone.gemfire.cache.snapshot.RegionSnapshotService;
-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.wan.GatewaySender;
 import com.gemstone.gemfire.compression.Compressor;
 import com.gemstone.gemfire.distributed.DistributedMember;
@@ -486,9 +483,6 @@ public abstract class AbstractRegion implements Region, RegionAttributes,
    */
   public CacheLoader basicGetLoader() {
     CacheLoader result = this.cacheLoader;
-    if (isBridgeLoader(result)) {
-      result = null;
-    }
     return result;
   }
   /**
@@ -498,9 +492,6 @@ public abstract class AbstractRegion implements Region, RegionAttributes,
    */
   public CacheWriter basicGetWriter() {
     CacheWriter result = this.cacheWriter;
-    if (isBridgeWriter(result)) {
-      result = null;
-    }
     return result;
   }
   
@@ -1200,11 +1191,6 @@ public abstract class AbstractRegion implements Region, RegionAttributes,
   // synchronized so not reentrant
   public synchronized CacheLoader setCacheLoader(CacheLoader cl) {
     checkReadiness();
-    if (cl != null && isBridgeLoader(cl)) {
-      if (getPoolName() != null) {
-        throw new IllegalStateException("A region with a connection pool can not have a BridgeLoader.");
-      }
-    }
     CacheLoader oldLoader = this.cacheLoader;
     assignCacheLoader(cl);
     cacheLoaderChanged(oldLoader);
@@ -1213,24 +1199,12 @@ public abstract class AbstractRegion implements Region, RegionAttributes,
 
   private synchronized void assignCacheLoader(CacheLoader cl) {
     this.cacheLoader = cl;
-    if (cl instanceof BridgeLoader) {
-      BridgeLoader bl = (BridgeLoader) cl;
-      bl.attach(this);
-    } else if (cl instanceof BridgeClient) {
-      BridgeClient bc = (BridgeClient)cl;
-      bc.attach(this);
-    }
   }
 
   // synchronized so not reentrant
   public synchronized CacheWriter setCacheWriter(CacheWriter cacheWriter)
   {
     checkReadiness();
-    if (cacheWriter != null && isBridgeWriter(cacheWriter)) {
-      if (getPoolName() != null) {
-        throw new IllegalStateException("A region with a connection pool can not have a BridgeWriter.");
-      }
-    }
     CacheWriter oldWriter = this.cacheWriter;
     assignCacheWriter(cacheWriter);
     cacheWriterChanged(oldWriter);
@@ -1240,10 +1214,6 @@ public abstract class AbstractRegion implements Region, RegionAttributes,
   private synchronized void assignCacheWriter(CacheWriter cacheWriter)
   {
     this.cacheWriter = cacheWriter;
-    if (cacheWriter instanceof BridgeWriter) {
-      BridgeWriter bw = (BridgeWriter)cacheWriter;
-      bw.attach(this);
-    }
   }
 
   void checkEntryTimeoutAction(String mode, ExpirationAction ea) {
@@ -1572,15 +1542,6 @@ public abstract class AbstractRegion implements Region, RegionAttributes,
   protected void closeCacheCallback(CacheCallback cb)
   {
     if (cb != null) {
-      if (cb instanceof BridgeWriter) {
-        BridgeWriter bw = (BridgeWriter)cb;
-        bw.detach(this);
-      }
-      else if (cb instanceof BridgeLoader) {
-        BridgeLoader bl = (BridgeLoader)cb;
-        bl.detach(this);
-      }
-
       try {
         cb.close();
       }
@@ -1610,19 +1571,6 @@ public abstract class AbstractRegion implements Region, RegionAttributes,
     // nothing needed by default
   }
 
-  /**
-   * @since 5.7
-   */
-  public static boolean isBridgeLoader(CacheLoader cl) {
-    return cl instanceof BridgeLoader || cl instanceof BridgeClient;
-  }
-  /**
-   * @since 5.7
-   */
-  public static boolean isBridgeWriter(CacheWriter cw) {
-    return cw instanceof BridgeWriter;
-  }
-
   protected void cacheWriterChanged(CacheWriter oldWriter)
   {
     if (this.cacheWriter != oldWriter) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserver.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserver.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserver.java
deleted file mode 100755
index d8b41a2..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserver.java
+++ /dev/null
@@ -1,89 +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;
-
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
-
-/**
- * This interface is used by testing/debugging code to be notified of different
- * events. See the documentation for class BridgeObserverHolder for details.
- * 
- * @author Yogesh Mahajan
- * @since 5.1
- *  
- */
-public interface BridgeObserver
-{
-  /**
-   * This callback is called when now primary Ep is identified.
-   */
-  public void afterPrimaryIdentificationFromBackup(ServerLocation location);
-
-  /**
-   * This callback is called just before interest registartion
-   */
-  public void beforeInterestRegistration();
-
-  /**
-   * This callback is called just after interest registartion
-   */
-  public void afterInterestRegistration();
-
-  /**
-   * This callback is called just before primary identification
-   */
-  public void beforePrimaryIdentificationFromBackup();
-
-  /**
-   * This callback is called just before Interest Recovery by DSM thread happens
-   */
-  public void beforeInterestRecovery();
-  
-  /**
-   * Invoked by CacheClientUpdater just before invoking endpointDied for
-   * fail over
-   * @param location ServerLocation which has failed
-   */
-  public void beforeFailoverByCacheClientUpdater(ServerLocation location);
-  /**
-   * Invoked before sending an instantiator message to server
-   * 
-   * @param eventId
-   */
-  public void beforeSendingToServer(EventID eventId);
-  /**
-   * Invoked after sending an instantiator message to server 
-   * 
-   * @param eventId
-   */
-  public void afterReceivingFromServer(EventID eventId);
-
-  /**
-   * This callback is called just before sending client ack to the primary servrer.
-   */
-   public void beforeSendingClientAck();  
-
-   /**
-    * Invoked after Message is created
-    *
-    * @param msg
-    */
-   public void afterMessageCreation(Message msg);
-   
-   /**
-    * Invoked after Queue Destroy Message has been sent
-    */
-   public void afterQueueDestroyMessage();
-   
-   /**
-    * Invoked after a primary is recovered from a backup or new connection. 
-    */
-   public void afterPrimaryRecovered(ServerLocation location);
-   
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserverAdapter.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserverAdapter.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserverAdapter.java
deleted file mode 100755
index 0dce187..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserverAdapter.java
+++ /dev/null
@@ -1,107 +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;
-
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
-
-/**
- * This class provides 'do-nothing' implementations of all of the methods of
- * interface BridgeObserver. See the documentation for class
- * BridgeObserverHolder for details.
- * 
- * @author Yogesh Mahajan
- * @since 5.1
- */
-public class BridgeObserverAdapter implements BridgeObserver
-{
-  /**
-   * This callback is called when now primary Ep is identified.
-   */
-  public void afterPrimaryIdentificationFromBackup(ServerLocation primaryEndpoint)
-  {
-  }
-
-  /**
-   * This callback is called just before interest registartion
-   */
-  public void beforeInterestRegistration()
-  {
-  }
-
-  /**
-   * This callback is called just after interest registartion
-   */
-  public void afterInterestRegistration()
-  {
-  }
-
-  /**
-   * This callback is called just before primary identification
-   */
-  public void beforePrimaryIdentificationFromBackup()
-  {
-  }
-
-  /**
-   * This callback is called just before Interest Recovery by DSM thread happens
-   */
-  public void beforeInterestRecovery()
-  {
-
-  }
-
-  public void beforeFailoverByCacheClientUpdater(ServerLocation epFailed)
-  {
-  }
-  /**
-   * Invoked before sending an instantiator message to server
-   * 
-   * @param eventId
-   */
-  public void beforeSendingToServer(EventID eventId){
-    
-  }
-  /**
-   * Invoked after sending an instantiator message to server 
-   * 
-   * @param eventId
-   */
-  public void afterReceivingFromServer(EventID eventId){
-    
-  }
-  
-  /**
-   * This callback is called just before sending client ack to the primary servrer.
-   */
-  public void beforeSendingClientAck(){
-    
-  }  
-
-  /**
-   * Invoked after Message is created
-   *
-   * @param msg
-   */
-  public void afterMessageCreation(Message msg){
-  
-  }
-  
-  /**
-   * Invoked after Queue Destroy Message has been sent
-   */
-  public void afterQueueDestroyMessage(){
-    
-  }
-  
-  /**
-   * Invoked after a primary is recovered from a backup or new connection. 
-   */
-  public void afterPrimaryRecovered(ServerLocation location) {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserverHolder.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserverHolder.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserverHolder.java
deleted file mode 100755
index d68a608..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeObserverHolder.java
+++ /dev/null
@@ -1,53 +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;
-
-import com.gemstone.gemfire.cache.query.internal.Support;
-
-/**
- * This class is intended to hold a single 'observer' which will receive
- * callbacks. There can be only one such observer at a time. If no observer is
- * needed, this member variable should point to an object with 'do-nothing'
- * methods, such as BridgeObserverAdapter.
- * 
- * @author Yogesh Mahajan
- * @since 5.1
- */
-public class BridgeObserverHolder
-  {
-
-  /**
-   * The default 'do-nothing' bridge observer *
-   */
-  private static final BridgeObserver NO_OBSERVER = new BridgeObserverAdapter();
-
-  /**
-   * The current observer which will be notified of all query events.
-   */
-  private static BridgeObserver _instance = NO_OBSERVER;
-
-  /**
-   * Set the given observer to be notified of events. Returns the current
-   * observer.
-   */
-  public static final BridgeObserver setInstance(BridgeObserver observer)
-  {
-    Support.assertArg(observer != null,
-        "setInstance expects a non-null argument!");
-    BridgeObserver oldObserver = _instance;
-    _instance = observer;
-    return oldObserver;
-  }
-
-  /** Return the current BridgeObserver instance */
-  public static final BridgeObserver getInstance()
-  {
-    return _instance;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeRegionEventImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeRegionEventImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeRegionEventImpl.java
deleted file mode 100755
index 7bcaf64..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeRegionEventImpl.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-
-package com.gemstone.gemfire.internal.cache;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.cache.Operation;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
-
-//import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
-
-/**
- * Class <code>BridgeRegionEventImpl</code> is an implementation of a bridge
- * region event, which is just an <code>RegionEvent</code> with the client's
- * host and port for notification purposes.
- * 
- * @author Girish Thombare
- * 
- * @since 5.1
- */
-public final class BridgeRegionEventImpl extends RegionEventImpl
-  {
-
-  /**
-   * The originating membershipId of this event.
-   */
-  private  ClientProxyMembershipID context;
-
-  public BridgeRegionEventImpl() {
-  }
-  
-  /**
-   * To be called from the Distributed Message without setting EventID
-   * @param region
-   * @param op
-   * @param callbackArgument
-   * @param originRemote
-   * @param distributedMember
-   */
-  public BridgeRegionEventImpl(LocalRegion region, Operation op, Object callbackArgument,boolean originRemote, DistributedMember distributedMember,ClientProxyMembershipID contx) {
-    super(region, op,callbackArgument, originRemote,distributedMember);
-    setContext(contx);
-  }
-
-  public BridgeRegionEventImpl(LocalRegion region, Operation op, Object callbackArgument,boolean originRemote, DistributedMember distributedMember,ClientProxyMembershipID contx,EventID eventId) {
-      super(region, op,callbackArgument, originRemote,distributedMember, eventId);
-      setContext(contx);
-  }
-
-
-  /**
-   * sets The membershipId originating this event
-   *  
-   */
-  protected void setContext(ClientProxyMembershipID contx)
-  {
-    this.context = contx;
-  }
-
-  /**
-   * Returns The context originating this event
-   * 
-   * @return The context originating this event
-   */
-  @Override
-  public ClientProxyMembershipID getContext()
-  {
-    return this.context;
-  }
-
-  @Override
-  public String toString()
-  {
-    String superStr = super.toString();
-    StringBuffer buffer = new StringBuffer();
-    String str = superStr.substring(0, superStr.length() - 1);
-    buffer.append(str).append(";context=").append(getContext()).append(']');
-    return buffer.toString();
-  }
-
-  @Override
-  public int getDSFID() {
-    return BRIDGE_REGION_EVENT;
-  }
-
-  @Override
-  public void toData(DataOutput out) throws IOException
-  {
-    super.toData(out);
-    DataSerializer.writeObject(getContext(), out);
-  }
-
-  @Override
-  public void fromData(DataInput in) throws IOException, ClassNotFoundException
-  {
-    super.fromData(in);
-    setContext(ClientProxyMembershipID.readCanonicalized(in));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeServerAdvisor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeServerAdvisor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeServerAdvisor.java
deleted file mode 100644
index 733b762..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeServerAdvisor.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2002-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
- * more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-
-package com.gemstone.gemfire.internal.cache;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.cache.server.ServerLoad;
-import com.gemstone.gemfire.distributed.internal.DistributionAdvisee;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-
-
-/**
- * Used to give advise to a bridge server.
- * Bridge server currently need to know about controller's
- * @author darrel
- *
- */
-public class BridgeServerAdvisor extends GridAdvisor {
-  
-  /** Creates a new instance of BridgeServerAdvisor */
-  private BridgeServerAdvisor(DistributionAdvisee server) {
-    super(server);
-  }
-
-  public static BridgeServerAdvisor createBridgeServerAdvisor(DistributionAdvisee server) {
-    BridgeServerAdvisor advisor = new BridgeServerAdvisor(server);
-    advisor.initialize();
-    return advisor;
-  }
-
-  @Override
-  public String toString() {
-    return "BridgeServerAdvisor for " + getAdvisee().getFullPath();
-  }
-
-  /** Instantiate new distribution profile for this member */
-  @Override
-  protected Profile instantiateProfile(
-      InternalDistributedMember memberId, int version) {
-    return new BridgeServerProfile(memberId, version);
-  }
-  
-  /**
-   * Describes a bridge server for distribution purposes.
-   */
-  public static class BridgeServerProfile extends GridAdvisor.GridProfile {
-    private String[] groups;
-    private int maxConnections;
-    private ServerLoad initialLoad;
-    private long loadPollInterval;
-
-    /** for internal use, required for DataSerializer.readObject */
-    public BridgeServerProfile() {
-    }
-
-    public BridgeServerProfile(InternalDistributedMember memberId, int version) {
-      super(memberId, version);
-    }
-
-    public BridgeServerProfile(BridgeServerProfile toCopy) {
-      super(toCopy);
-      this.groups = toCopy.groups;
-    }
-
-    /** don't modify the returned array! */
-    public String[] getGroups() {
-      return this.groups;
-    }
-    public void setGroups(String[] groups) {
-      this.groups = groups;
-    }
-    
-    public ServerLoad getInitialLoad() {
-      return initialLoad;
-    }
-    
-    public int getMaxConnections() {
-      return maxConnections;
-    }
-    
-    public void setMaxConnections(int maxConnections) {
-      this.maxConnections = maxConnections;
-    }
-
-    public void setInitialLoad(ServerLoad initialLoad) {
-      this.initialLoad = initialLoad;
-    }
-    public long getLoadPollInterval() {
-      return this.loadPollInterval;
-    }
-    public void setLoadPollInterval(long v) {
-      this.loadPollInterval = v;
-    }
-
-    /**
-     * Used to process an incoming bridge server profile. Any controller in this
-     * vm needs to be told about this incoming new bridge server. The reply
-     * needs to contain any controller(s) that exist in this vm.
-     * 
-     * @since 5.7
-     */
-    @Override
-    public void processIncoming(DistributionManager dm, String adviseePath,
-        boolean removeProfile, boolean exchangeProfiles,
-        final List<Profile> replyProfiles) {
-      // tell local controllers about this bridge server
-      tellLocalControllers(removeProfile, exchangeProfiles, replyProfiles);
-      // for QRM messaging we need bridge servers to know about each other
-      tellLocalBridgeServers(removeProfile, exchangeProfiles, replyProfiles);
-    }
-
-    @Override
-    public int getDSFID() {
-      return BRIDGE_SERVER_PROFILE;
-    }
-
-    @Override
-    public void toData(DataOutput out) throws IOException {
-      super.toData(out);
-      DataSerializer.writeStringArray(this.groups, out);
-      out.writeInt(maxConnections);
-      InternalDataSerializer.invokeToData(initialLoad, out);
-      out.writeLong(getLoadPollInterval());
-    }
-
-    @Override
-    public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-      super.fromData(in);
-      this.groups = DataSerializer.readStringArray(in);
-      this.maxConnections = in.readInt();
-      this.initialLoad = new ServerLoad();
-      InternalDataSerializer.invokeFromData(initialLoad, in);
-      setLoadPollInterval(in.readLong());
-    }
-
-    @Override
-    public StringBuilder getToStringHeader() {
-      return new StringBuilder("BridgeServerProfile");
-    }
-
-    @Override
-    public void fillInToString(StringBuilder sb) {
-      super.fillInToString(sb);
-      if (this.groups != null) {
-        sb.append("; groups=" + Arrays.asList(this.groups));
-        sb.append("; maxConnections=" + maxConnections);
-        sb.append("; initialLoad=" + initialLoad);
-        sb.append("; loadPollInterval=" + getLoadPollInterval());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeServerImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeServerImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeServerImpl.java
deleted file mode 100644
index f7fa15f..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BridgeServerImpl.java
+++ /dev/null
@@ -1,816 +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;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.CancelCriterion;
-import com.gemstone.gemfire.GemFireIOException;
-import com.gemstone.gemfire.InternalGemFireError;
-import com.gemstone.gemfire.InvalidValueException;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.ClientSession;
-import com.gemstone.gemfire.cache.DataPolicy;
-import com.gemstone.gemfire.cache.DiskStore;
-import com.gemstone.gemfire.cache.DiskStoreFactory;
-import com.gemstone.gemfire.cache.DynamicRegionFactory;
-import com.gemstone.gemfire.cache.EvictionAction;
-import com.gemstone.gemfire.cache.ExpirationAction;
-import com.gemstone.gemfire.cache.ExpirationAttributes;
-import com.gemstone.gemfire.cache.InterestRegistrationListener;
-import com.gemstone.gemfire.cache.RegionAttributes;
-import com.gemstone.gemfire.cache.RegionExistsException;
-import com.gemstone.gemfire.cache.Scope;
-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.server.internal.LoadMonitor;
-import com.gemstone.gemfire.cache.util.BridgeMembership;
-import com.gemstone.gemfire.cache.util.BridgeMembershipListener;
-import com.gemstone.gemfire.cache.wan.GatewayTransportFilter;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.distributed.internal.DM;
-import com.gemstone.gemfire.distributed.internal.DistributionAdvisee;
-import com.gemstone.gemfire.distributed.internal.DistributionAdvisor;
-import com.gemstone.gemfire.distributed.internal.DistributionAdvisor.Profile;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.distributed.internal.ResourceEvent;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
-import com.gemstone.gemfire.internal.Assert;
-import com.gemstone.gemfire.internal.OSProcess;
-import com.gemstone.gemfire.internal.admin.ClientHealthMonitoringRegion;
-import com.gemstone.gemfire.internal.cache.BridgeServerAdvisor.BridgeServerProfile;
-import com.gemstone.gemfire.internal.cache.ha.HARegionQueue;
-import com.gemstone.gemfire.internal.cache.tier.Acceptor;
-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.ClientProxyMembershipID;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
-
-/**
- * An implementation of the <code>CacheServer</code> interface that delegates
- * most of the heavy lifting to an {@link Acceptor}.
- * 
- * @author David Whitlock
- * @since 4.0
- */
-@SuppressWarnings("deprecation")
-public class BridgeServerImpl
-  extends AbstractBridgeServer
-  implements DistributionAdvisee {
-
-  private static final Logger logger = LogService.getLogger();
-  
-  private static final int FORCE_LOAD_UPDATE_FREQUENCY= Integer.getInteger("gemfire.BridgeServer.FORCE_LOAD_UPDATE_FREQUENCY", 10).intValue();
-  
-  /** The acceptor that does the actual serving */
-  private volatile AcceptorImpl acceptor;
-
-  // moved to AbstractBridgeServer
-
-
-
-  /**
-   * The advisor used by this bridge sever.
-   * @since 5.7
-   */
-  private volatile BridgeServerAdvisor advisor;
-
-  /**
-   * The monitor used to monitor load on this
-   * bridge server and distribute load to the locators
-   * @since 5.7
-   */
-  private volatile LoadMonitor loadMonitor;
-
-  /**
-   * boolean that represents whether this server is a GatewayReceiver or a simple BridgeServer
-   */
-  private boolean isGatewayReceiver;
-  
-  private List<GatewayTransportFilter> gatewayTransportFilters = Collections.EMPTY_LIST;
-  
-  /**
-   * Needed because this guy is an advisee
-   * @since 5.7
-   */
-  private int serialNumber; // changed on each start
-
-  public static final boolean ENABLE_NOTIFY_BY_SUBSCRIPTION_FALSE = 
-  Boolean.getBoolean("gemfire.cache-server.enable-notify-by-subscription-false");
-  
- 
-  // ////////////////////// Constructors //////////////////////
-
-  /**
-   * Creates a new <code>BridgeServerImpl</code> that serves the contents of
-   * the give <code>Cache</code>. It has the default configuration.
-   */
-  public BridgeServerImpl(GemFireCacheImpl cache, boolean isGatewayReceiver) {
-    super(cache);
-    this.isGatewayReceiver = isGatewayReceiver;
-  }
-
-  // //////////////////// Instance Methods ///////////////////
-  
-  public CancelCriterion getCancelCriterion() {
-    return cache.getCancelCriterion();    
-  }
-
-  /**
-   * Checks to see whether or not this bridge server is running. If so, an
-   * {@link IllegalStateException} is thrown.
-   */
-  private void checkRunning() {
-    if (this.isRunning()) {
-      throw new IllegalStateException(LocalizedStrings.BridgeServerImpl_A_BRIDGE_SERVERS_CONFIGURATION_CANNOT_BE_CHANGED_ONCE_IT_IS_RUNNING.toLocalizedString());
-    }
-  }
-
-  public boolean isGatewayReceiver() {
-    return this.isGatewayReceiver;
-  }
-  
-  @Override
-  public int getPort() {
-    if (this.acceptor != null) {
-      return this.acceptor.getPort();
-    }
-    else {
-      return super.getPort();
-    }
-  }
-
-  @Override
-  public void setPort(int port) {
-    checkRunning();
-    super.setPort(port);
-  }
-
-  @Override
-  public void setBindAddress(String address) {
-    checkRunning();
-    super.setBindAddress(address);
-  }
-  @Override
-  public void setHostnameForClients(String name) {
-    checkRunning();
-    super.setHostnameForClients(name);
-  }
-
-  @Override
-  public void setMaxConnections(int maxCon) {
-    checkRunning();
-    super.setMaxConnections(maxCon);
-  }
-
-  @Override
-  public void setMaxThreads(int maxThreads) {
-    checkRunning();
-    super.setMaxThreads(maxThreads);
-  }
-
-  @Override
-  public void setNotifyBySubscription(boolean b) {
-    checkRunning();
-    if (BridgeServerImpl.ENABLE_NOTIFY_BY_SUBSCRIPTION_FALSE) {
-      this.notifyBySubscription = b;
-    }
-  }
-
-  @Override
-  public void setMaximumMessageCount(int maximumMessageCount) {
-    checkRunning();
-    super.setMaximumMessageCount(maximumMessageCount);
-  }
-
-  @Override
-  public void setSocketBufferSize(int socketBufferSize) {
-    this.socketBufferSize = socketBufferSize;
-  }
-
-  @Override
-  public int getSocketBufferSize() {
-    return this.socketBufferSize;
-  }
-  
-  @Override
-  public void setMaximumTimeBetweenPings(int maximumTimeBetweenPings) {
-    this.maximumTimeBetweenPings = maximumTimeBetweenPings;
-  }
-
-  @Override
-  public int getMaximumTimeBetweenPings() {
-    return this.maximumTimeBetweenPings;
-  }
-
-
-  @Override
-  public void setLoadPollInterval(long loadPollInterval) {
-    checkRunning();
-    super.setLoadPollInterval(loadPollInterval);
-  }
-
-  @Override
-  public int getMaximumMessageCount() {
-    return this.maximumMessageCount;
-  }
-
-  @Override
-  public void setLoadProbe(ServerLoadProbe loadProbe) {
-    checkRunning();
-    super.setLoadProbe(loadProbe);
-  }
-
-  public void setGatewayTransportFilter(
-      List<GatewayTransportFilter> transportFilters) {
-    this.gatewayTransportFilters = transportFilters;
-  }
-  
-  @Override
-  public int getMessageTimeToLive() {
-    return this.messageTimeToLive;
-  }
-  
-
-  public ClientSubscriptionConfig getClientSubscriptionConfig(){
-    return this.clientSubscriptionConfig;
-  }
-
-  /**
-   * Sets the configuration of <b>this</b> <code>CacheServer</code> based on
-   * the configuration of <b>another</b> <code>CacheServer</code>.
-   */
-  public void configureFrom(CacheServer other) {
-    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());
-    }
-  }
-
-  @Override
-  public synchronized void start() throws IOException {
-    Assert.assertTrue(this.cache != null);
-    boolean isSqlFabricSystem = ((GemFireCacheImpl)this.cache).isSqlfSystem();
-    
-    this.serialNumber = createSerialNumber();
-    if (DynamicRegionFactory.get().isOpen()) {
-      // force notifyBySubscription to be true so that meta info is pushed
-      // from servers to clients instead of invalidates.
-      if (!this.notifyBySubscription) {
-        logger.info(LocalizedMessage.create(LocalizedStrings.BridgeServerImpl_FORCING_NOTIFYBYSUBSCRIPTION_TO_SUPPORT_DYNAMIC_REGIONS));
-        this.notifyBySubscription = true;
-      }
-    }
-    this.advisor = BridgeServerAdvisor.createBridgeServerAdvisor(this);
-    this.loadMonitor = new LoadMonitor(loadProbe, maxConnections,
-        loadPollInterval, FORCE_LOAD_UPDATE_FREQUENCY, 
-        advisor);
-    List overflowAttributesList = new LinkedList();
-    ClientSubscriptionConfig csc = this.getClientSubscriptionConfig();
-    overflowAttributesList.add(0, csc.getEvictionPolicy());
-    overflowAttributesList.add(1, Integer.valueOf(csc.getCapacity()));
-    overflowAttributesList.add(2, Integer.valueOf(this.port));
-    String diskStoreName = csc.getDiskStoreName();
-    if (diskStoreName != null) {
-      overflowAttributesList.add(3, diskStoreName);
-      overflowAttributesList.add(4, true); // indicator to use diskstore
-    } else {
-      overflowAttributesList.add(3, csc.getOverflowDirectory());
-      overflowAttributesList.add(4, false);
-    }
-
-    this.acceptor = new AcceptorImpl(getPort(), 
-                                     getBindAddress(),
-                                     getNotifyBySubscription(),
-                                     getSocketBufferSize(), 
-                                     getMaximumTimeBetweenPings(), 
-                                     this.cache,
-                                     getMaxConnections(), 
-                                     getMaxThreads(), 
-                                     getMaximumMessageCount(),
-                                     getMessageTimeToLive(),
-                                     getTransactionTimeToLive(),
-                                     this.loadMonitor,
-                                     overflowAttributesList, 
-                                     isSqlFabricSystem,
-                                     this.isGatewayReceiver,
-                                     this.gatewayTransportFilters, this.tcpNoDelay);
-
-    this.acceptor.start();
-    this.advisor.handshake();
-    this.loadMonitor.start(new ServerLocation(getExternalAddress(),
-        getPort()), acceptor.getStats());
-    
-    // TODO : Need to provide facility to enable/disable client health monitoring.
-    //Creating ClientHealthMonitoring region.
-    // Force initialization on current cache
-    if(cache instanceof GemFireCacheImpl) {
-      ClientHealthMonitoringRegion.getInstance((GemFireCacheImpl)cache);
-    }
-    this.cache.getLoggerI18n().config(LocalizedStrings.BridgeServerImpl_CACHESERVER_CONFIGURATION___0, getConfig());
-    
-    /* 
-     * If the stopped bridge server is restarted, we'll need to re-register the 
-     * client membership listener. If the listener is already registered it 
-     * won't be registered as would the case when start() is invoked for the 
-     * first time.  
-     */
-    BridgeMembershipListener[] membershipListeners = 
-                                BridgeMembership.getBridgeMembershipListeners();
-    
-    boolean membershipListenerRegistered = false;
-    for (BridgeMembershipListener membershipListener : membershipListeners) {
-      //just checking by reference as the listener instance is final
-      if (listener == membershipListener) {
-        membershipListenerRegistered = true;
-        break;
-      }
-    }
-    
-    if (!membershipListenerRegistered) {
-      BridgeMembership.registerBridgeMembershipListener(listener);
-    }
-    
-    if (!isGatewayReceiver) {
-      InternalDistributedSystem system = ((GemFireCacheImpl) this.cache)
-          .getDistributedSystem();
-      system.handleResourceEvent(ResourceEvent.CACHE_SERVER_START, this);
-    }
-    
-  }
-
-  
-  /**
-   * Gets the address that this bridge server can be contacted on from external
-   * processes.
-   * @since 5.7
-   */
-  public String getExternalAddress() {
-    return getExternalAddress(true);
-  }
-  
-  public String getExternalAddress(boolean checkServerRunning) {
-    if (checkServerRunning) {
-      if (!this.isRunning()) {
-        String s = "A bridge server's bind address is only available if it has been started";
-        this.cache.getCancelCriterion().checkCancelInProgress(null);
-        throw new IllegalStateException(s);
-      }
-    }
-    if (this.hostnameForClients == null || this.hostnameForClients.equals("")) {
-      if (this.acceptor != null) {
-        return this.acceptor.getExternalAddress();
-      }
-      else {
-        return null;
-      }
-    }
-    else {
-      return this.hostnameForClients;
-    }
-  }
-
-  public boolean isRunning() {
-    return this.acceptor != null && this.acceptor.isRunning();
-  }
-
-  public synchronized void stop() {
-    if (!isRunning()) {
-      return;
-    }
-    
-    RuntimeException firstException = null;
-    
-    try {
-      if(this.loadMonitor != null) {
-        this.loadMonitor.stop();
-      }
-    } catch(RuntimeException e) {
-      cache.getLoggerI18n().warning(LocalizedStrings.BridgeServerImpl_CACHESERVER_ERROR_CLOSING_LOAD_MONITOR, e);
-      firstException = e;
-    }
-    
-    try {
-      if (this.advisor != null) {
-        this.advisor.close();
-      }
-    } catch(RuntimeException e) {
-      cache.getLoggerI18n().warning(LocalizedStrings.BridgeServerImpl_CACHESERVER_ERROR_CLOSING_ADVISOR, e);
-      firstException = e;
-    } 
-    
-    try {
-      if (this.acceptor != null) {
-        this.acceptor.close();
-      }
-    } catch(RuntimeException e) {
-      logger.warn(LocalizedMessage.create(LocalizedStrings.BridgeServerImpl_CACHESERVER_ERROR_CLOSING_ACCEPTOR_MONITOR), e);
-      if (firstException != null) {
-        firstException = e;
-      }
-    }
-    
-    if(firstException != null) {
-      throw firstException;
-    }
-    
-    //TODO : We need to clean up the admin region created for client
-    //monitoring.
-    
-    // BridgeServer is still available, just not running, so we don't take
-    // it out of the cache's list...
-    // cache.removeBridgeServer(this);
-
-    /* Assuming start won't be called after stop */
-    BridgeMembership.unregisterBridgeMembershipListener(this.listener);
-    
-    TXManagerImpl txMgr = (TXManagerImpl) cache.getCacheTransactionManager();
-    txMgr.removeHostedTXStatesForClients();
-    
-    if (!isGatewayReceiver) {
-      InternalDistributedSystem system = ((GemFireCacheImpl) this.cache)
-          .getDistributedSystem();
-      system.handleResourceEvent(ResourceEvent.CACHE_SERVER_STOP, this);
-    }
-
-  }
-
-  private String getConfig() {
-    ClientSubscriptionConfig csc = this.getClientSubscriptionConfig();
-    String str =
-    "port=" + getPort() + " max-connections=" + getMaxConnections()
-        + " max-threads=" + getMaxThreads() + " notify-by-subscription="
-        + getNotifyBySubscription() + " socket-buffer-size="
-        + getSocketBufferSize() + " maximum-time-between-pings="
-        + getMaximumTimeBetweenPings() + " maximum-message-count="
-        + getMaximumMessageCount() + " message-time-to-live="
-        + getMessageTimeToLive() + " eviction-policy=" + csc.getEvictionPolicy()
-        + " capacity=" + csc.getCapacity() + " overflow directory=";
-    if (csc.getDiskStoreName() != null) {
-      str += csc.getDiskStoreName();
-    } else {
-      str += csc.getOverflowDirectory(); 
-    }
-    str += 
-        " groups=" + Arrays.asList(getGroups())
-        + " loadProbe=" + loadProbe
-        + " loadPollInterval=" + loadPollInterval
-        + " tcpNoDelay=" + tcpNoDelay;
-    return str;
-  }
-
-  @Override
-  public String toString() {
-    ClientSubscriptionConfig csc = this.getClientSubscriptionConfig();
-    String str = 
-    "CacheServer on port=" + getPort() + " client subscription config policy="
-        + csc.getEvictionPolicy() + " client subscription config capacity="
-        + csc.getCapacity();
-    if (csc.getDiskStoreName() != null) {
-      str += " client subscription config overflow disk store="
-        + csc.getDiskStoreName();
-    } else {
-      str += " client subscription config overflow directory="
-        + csc.getOverflowDirectory();
-    }
-    return str;
-  }
-
-  /**
-   * Test method used to access the internal acceptor
-   * 
-   * @return the internal acceptor
-   */
-  public AcceptorImpl getAcceptor() {
-    return this.acceptor;
-  }
-
-  // DistributionAdvisee methods
-
-  public DM getDistributionManager() {
-    return getSystem().getDistributionManager();
-  }
-  
-  public ClientSession getClientSession(String durableClientId) {
-    return getCacheClientNotifier().getClientProxy(durableClientId);
-  }
-
-  public ClientSession getClientSession(DistributedMember member) {
-    return getCacheClientNotifier().getClientProxy(
-        ClientProxyMembershipID.getClientId(member));
-  }
-  
-  public Set getAllClientSessions() {
-    return new HashSet(getCacheClientNotifier().getClientProxies());
-  }
-
-  /**
-   * create client subscription
-   * 
-   * @param cache
-   * @param ePolicy
-   * @param capacity
-   * @param port
-   * @param overFlowDir
-   * @param isDiskStore
-   * @return client subscription name
-   * @since 5.7
-   */
-  public static String clientMessagesRegion(GemFireCacheImpl cache, String ePolicy,
-      int capacity, int port, String overFlowDir, boolean isDiskStore) {
-    AttributesFactory factory = getAttribFactoryForClientMessagesRegion(cache, 
-        ePolicy, capacity, overFlowDir, isDiskStore);
-    RegionAttributes attr = factory.create();
-
-    return createClientMessagesRegion(attr, cache, capacity, port);
-  }
-
-  public static AttributesFactory getAttribFactoryForClientMessagesRegion(
-      GemFireCacheImpl cache,
-      String ePolicy, int capacity, String overflowDir, boolean isDiskStore)
-      throws InvalidValueException, GemFireIOException {
-    AttributesFactory factory = new AttributesFactory();
-    factory.setScope(Scope.LOCAL);
-
-    if (isDiskStore) {
-      // overflowDir parameter is actually diskstore name
-      factory.setDiskStoreName(overflowDir);
-      // client subscription queue is always overflow to disk, so do async
-      // see feature request #41479
-      factory.setDiskSynchronous(true);
-    } else if  (overflowDir == null || overflowDir.equals(ClientSubscriptionConfig.DEFAULT_OVERFLOW_DIRECTORY)) {
-      factory.setDiskStoreName(null);
-      // client subscription queue is always overflow to disk, so do async
-      // see feature request #41479
-      factory.setDiskSynchronous(true);
-    } else {
-      File dir = new File(overflowDir + File.separatorChar
-          + generateNameForClientMsgsRegion(OSProcess.getId()));
-      // This will delete the overflow directory when virtual machine terminates.
-      dir.deleteOnExit();
-      if (!dir.mkdirs() && !dir.isDirectory()) {
-        throw new GemFireIOException("Could not create client subscription overflow directory: "
-            + dir.getAbsolutePath());
-      }
-      File[] dirs = { dir };
-      DiskStoreFactory dsf = cache.createDiskStoreFactory();
-      DiskStore bsi = dsf.setAutoCompact(true)
-      .setDiskDirsAndSizes(dirs, new int[] { Integer.MAX_VALUE })
-      .create("bsi");
-      factory.setDiskStoreName("bsi");
-      // backward compatibility, it was sync
-      factory.setDiskSynchronous(true);
-    }
-    factory.setDataPolicy(DataPolicy.NORMAL);
-    // enable statistics
-    factory.setStatisticsEnabled(true);
-    /* setting LIFO related eviction attributes */
-    if (HARegionQueue.HA_EVICTION_POLICY_ENTRY.equals(ePolicy)) {
-      factory
-          .setEvictionAttributes(EvictionAttributesImpl.createLIFOEntryAttributes(
-              capacity, EvictionAction.OVERFLOW_TO_DISK));
-    }
-    else if (HARegionQueue.HA_EVICTION_POLICY_MEMORY.equals(ePolicy)) { // condition refinement
-      factory
-          .setEvictionAttributes(EvictionAttributesImpl.createLIFOMemoryAttributes(
-              capacity, EvictionAction.OVERFLOW_TO_DISK));
-    }
-    else {
-      // throw invalid eviction policy exception
-      throw new InvalidValueException(
-        LocalizedStrings.BridgeServerImpl__0_INVALID_EVICTION_POLICY.toLocalizedString(ePolicy));
-    }
-    return factory;
-  }
-
-  public static String createClientMessagesRegion(RegionAttributes attr,
-      GemFireCacheImpl cache, int capacity, int port) {
-    // generating unique name in VM for ClientMessagesRegion
-    String regionName = generateNameForClientMsgsRegion(port);
-    try {
-      cache.createVMRegion(regionName, attr,
-          new InternalRegionArguments().setDestroyLockFlag(true)
-              .setRecreateFlag(false).setSnapshotInputStream(null)
-              .setImageTarget(null).setIsUsedForMetaRegion(true));
-    }
-    catch (RegionExistsException ree) {
-      InternalGemFireError assErr = new InternalGemFireError(
-          "unexpected exception");
-      assErr.initCause(ree);
-      throw assErr;
-    }
-    catch (IOException e) {
-      // only if loading snapshot, not here
-      InternalGemFireError assErr = new InternalGemFireError(
-          "unexpected exception");
-      assErr.initCause(e);
-      throw assErr;
-    }
-    catch (ClassNotFoundException e) {
-      // only if loading snapshot, not here
-      InternalGemFireError assErr = new InternalGemFireError(
-          "unexpected exception");
-      assErr.initCause(e);
-      throw assErr;
-    }
-    return regionName;
-  }
-
-  public static String createClientMessagesRegionForTesting(GemFireCacheImpl cache,
-      String ePolicy, int capacity, int port, int expiryTime, String overFlowDir, boolean isDiskStore) {
-    AttributesFactory factory = getAttribFactoryForClientMessagesRegion(cache, 
-        ePolicy, capacity, overFlowDir, isDiskStore);
-    ExpirationAttributes ea = new ExpirationAttributes(expiryTime,
-        ExpirationAction.LOCAL_INVALIDATE);
-    factory.setEntryTimeToLive(ea);
-    RegionAttributes attr = factory.create();
-
-    return createClientMessagesRegion(attr, cache, capacity, port);
-  }
-
-  /**
-   * Generates the name for the client subscription using the given id.
-   * 
-   * @param id
-   * @return String
-   * @since 5.7 
-   */
-  public static String generateNameForClientMsgsRegion(int id) {
-    return ClientSubscriptionConfigImpl.CLIENT_SUBSCRIPTION + "_" + id;
-  }
-
-  /*
-   * Marker class name to identify the lock more easily in thread dumps private
-   * static class ClientMessagesRegionLock extends Object { }
-   */
-  public DistributionAdvisor getDistributionAdvisor() {
-    return this.advisor;
-  }
-  
-  /**
-   * Returns the BridgeServerAdvisor for this server
-   */
-  public BridgeServerAdvisor getCacheServerAdvisor() {
-    return this.advisor;
-  }
-  
-  public Profile getProfile() {
-    return getDistributionAdvisor().createProfile();
-  }
-  
-  public DistributionAdvisee getParentAdvisee() {
-    return null;
-  }
-  
-  /**
-   * Returns the underlying <code>InternalDistributedSystem</code> connection.
-   * @return the underlying <code>InternalDistributedSystem</code>
-   */
-  public InternalDistributedSystem getSystem() {
-    return (InternalDistributedSystem)this.cache.getDistributedSystem();
-  }
-  
-  public String getName() {
-    return "CacheServer";
-  }
-  
-  public String getFullPath() {
-    return getName();
-  }
-
-  private final static AtomicInteger profileSN = new AtomicInteger();
-  
-  private static int createSerialNumber() {
-    return profileSN.incrementAndGet();
-  }
-
-  /**
-   * Returns an array of all the groups of this bridge server.
-   * This includes those from the groups gemfire property
-   * and those explicitly added to this server.
-   */
-  public String[] getCombinedGroups() {
-    ArrayList<String> groupList = new ArrayList<String>();
-    for (String g: MemberAttributes.parseGroups(null, getSystem().getConfig().getGroups())) {
-      if (!groupList.contains(g)) {
-        groupList.add(g);
-      }
-    }
-    for (String g: getGroups()) {
-      if (!groupList.contains(g)) {
-        groupList.add(g);
-      }
-    }
-    String[] groups = new String[groupList.size()];
-    return groupList.toArray(groups);
-  }
-  
-  public /*synchronized causes deadlock*/ void fillInProfile(Profile profile) {
-    assert profile instanceof BridgeServerProfile;
-    BridgeServerProfile bp = (BridgeServerProfile)profile;
-    bp.setHost(getExternalAddress(false));
-    bp.setPort(getPort());
-    bp.setGroups(getCombinedGroups());
-    bp.setMaxConnections(maxConnections);
-    bp.setInitialLoad(loadMonitor.getLastLoad());
-    bp.setLoadPollInterval(getLoadPollInterval());
-    bp.serialNumber = getSerialNumber();
-    bp.finishInit();
-  }
-
-  public int getSerialNumber() {
-    return this.serialNumber;
-  }
-
-  
-   protected CacheClientNotifier getCacheClientNotifier() {
-    return getAcceptor().getCacheClientNotifier();
-  } 
-   
-  /**
-   * Registers a new <code>InterestRegistrationListener</code> with the set of
-   * <code>InterestRegistrationListener</code>s.
-   * 
-   * @param listener
-   *                The <code>InterestRegistrationListener</code> to register
-   * @throws IllegalStateException if the BridgeServer has not been started
-   * @since 5.8Beta
-   */
-  public void registerInterestRegistrationListener(
-      InterestRegistrationListener listener) {
-    if (!this.isRunning()) {
-      throw new IllegalStateException(LocalizedStrings.BridgeServerImpl_MUST_BE_RUNNING.toLocalizedString());
-    }
-    getCacheClientNotifier().registerInterestRegistrationListener(listener); 
-  }
-
-  /**
-   * Unregisters an existing <code>InterestRegistrationListener</code> from
-   * the set of <code>InterestRegistrationListener</code>s.
-   * 
-   * @param listener
-   *                The <code>InterestRegistrationListener</code> to
-   *                unregister
-   * 
-   * @since 5.8Beta
-   */
-  public void unregisterInterestRegistrationListener(
-      InterestRegistrationListener listener) {
-    getCacheClientNotifier().unregisterInterestRegistrationListener(listener);     
-  }
-
-  /**
-   * Returns a read-only set of <code>InterestRegistrationListener</code>s
-   * registered with this notifier.
-   * 
-   * @return a read-only set of <code>InterestRegistrationListener</code>s
-   *         registered with this notifier
-   * 
-   * @since 5.8Beta
-   */
-  public Set getInterestRegistrationListeners() {
-    return getCacheClientNotifier().getInterestRegistrationListeners(); 
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java
index 1299d75..ec75a92 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java
@@ -1789,7 +1789,7 @@ public final class BucketAdvisor extends CacheDistributionAdvisor  {
 
       HashSet<BucketServerLocation66> serverLocations = new HashSet<BucketServerLocation66>();
       for (Object object : servers) {
-        BridgeServerImpl server = (BridgeServerImpl)object;
+        CacheServerImpl server = (CacheServerImpl)object;
         if (server.isRunning() && (server.getExternalAddress() != null)) {
           BucketServerLocation66 location = new BucketServerLocation66(
               getBucket().getId(), server.getPort(), server

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java
index 7aaa241..4c29879 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java
@@ -9,7 +9,7 @@ package com.gemstone.gemfire.internal.cache;
 
 import java.util.List;
 
-import com.gemstone.gemfire.internal.cache.xmlcache.BridgeServerCreation;
+import com.gemstone.gemfire.internal.cache.xmlcache.CacheServerCreation;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.pdx.PdxSerializer;
 import com.gemstone.gemfire.pdx.ReflectionBasedAutoSerializer;
@@ -40,7 +40,7 @@ public class CacheConfig {
   /**
    * list of cache servers to create after auto-reconnect if cluster configuration is being used
    */
-  private List<BridgeServerCreation> cacheServerCreation;
+  private List<CacheServerCreation> cacheServerCreation;
   
   /**
    * This indicates if the pdxReadSerialized value is set by user. This is used 
@@ -137,12 +137,12 @@ public class CacheConfig {
   }
 
   
-  public List<BridgeServerCreation> getCacheServerCreation() {
+  public List<CacheServerCreation> getCacheServerCreation() {
     return this.cacheServerCreation;
   }
   
   
-  public void setCacheServerCreation(List<BridgeServerCreation> servers) {
+  public void setCacheServerCreation(List<CacheServerCreation> servers) {
     this.cacheServerCreation = servers;
   }
 


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

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
new file mode 100644
index 0000000..5dd3cfb
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipDUnitTest.java
@@ -0,0 +1,1642 @@
+/*=========================================================================
+ * 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.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import com.gemstone.gemfire.InternalGemFireException;
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.Statistics;
+import com.gemstone.gemfire.StatisticsType;
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.Pool;
+import com.gemstone.gemfire.cache.client.PoolManager;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.DurableClientAttributes;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+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.ServerConnection;
+import com.gemstone.gemfire.internal.logging.LocalLogWriter;
+import com.gemstone.gemfire.internal.logging.InternalLogWriter;
+import com.gemstone.gemfire.management.membership.ClientMembership;
+import com.gemstone.gemfire.management.membership.ClientMembershipEvent;
+import com.gemstone.gemfire.management.membership.ClientMembershipListener;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.SerializableRunnable;
+import dunit.VM;
+import dunit.DistributedTestCase.WaitCriterion;
+
+/**
+ * Tests the ClientMembership API including ClientMembershipListener.
+ *
+ * @author Kirk Lund
+ * @since 4.2.1
+ */
+public class ClientMembershipDUnitTest extends ClientServerTestCase {
+
+  protected static final boolean CLIENT = true;
+  protected static final boolean SERVER = false;
+  
+  protected static final int JOINED = 0;
+  protected static final int LEFT = 1;
+  protected static final int CRASHED = 2;
+    
+  public ClientMembershipDUnitTest(String name) {
+    super(name);
+  }
+
+  public void setUp() throws Exception {
+    super.setUp();
+    getSystem();
+  }
+  
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+    InternalClientMembership.unregisterAllListeners();
+  }
+
+  private void waitForAcceptsInProgressToBe(final int target)
+    throws Exception {
+    WaitCriterion ev = new WaitCriterion() {
+      String excuse;
+      public boolean done() {
+        int actual = getAcceptsInProgress();
+        if (actual == getAcceptsInProgress()) {
+          return true;
+        }
+        excuse = "accepts in progress (" + actual + ") never became " + target;
+        return false;
+      }
+      public String description() {
+        return excuse;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 60 * 1000, 200, true);
+  }
+  
+  protected int getAcceptsInProgress() {
+    StatisticsType st = InternalDistributedSystem.getAnyInstance().findType("CacheServerStats");
+    Statistics[] s = InternalDistributedSystem.getAnyInstance().findStatisticsByType(st);
+    return s[0].getInt("acceptsInProgress");
+  }
+
+  protected static Socket meanSocket;
+
+  /** test that a server times out waiting for a handshake that
+      never arrives. 
+   */
+  public void testConnectionTimeout() throws Exception {
+    addExpectedException("failed accepting client connection");
+    final Host host = Host.getHost(0);
+    final String hostName = getServerHostName(host);
+    final VM vm0 = host.getVM(0);
+    System.setProperty(AcceptorImpl.ACCEPT_TIMEOUT_PROPERTY_NAME, "1000");
+    try {
+    final int port = startBridgeServer(0);
+//    AsyncInvocation ai = null;
+    try {
+      assertTrue(port != 0);
+      SerializableRunnable createMeanSocket = new CacheSerializableRunnable("Connect to server with socket") {
+        public void run2() throws CacheException {
+          getCache(); // create a cache so we have stats
+          getLogWriter().info("connecting to cache server with socket");
+          try {
+            InetAddress addr = InetAddress.getByName(hostName);
+            meanSocket = new Socket(addr, port);
+          }
+          catch (Exception e) {
+            throw new RuntimeException("Test failed to connect or was interrupted", e);
+          }
+        }
+      };
+      SerializableRunnable closeMeanSocket = new CacheSerializableRunnable("close mean socket") {
+        public void run2() throws CacheException {
+          getLogWriter().info("closing mean socket");
+          try {
+            meanSocket.close();
+          }
+          catch (IOException ignore) {
+          }
+        }
+      };
+
+      assertEquals(0, getAcceptsInProgress());
+      
+      getLogWriter().info("creating mean socket");
+      vm0.invoke(createMeanSocket);
+      try {
+        getLogWriter().info("waiting to see it connect on server");
+        waitForAcceptsInProgressToBe(1);
+      } finally {
+        getLogWriter().info("closing mean socket");
+        vm0.invoke(closeMeanSocket);
+      }
+      getLogWriter().info("waiting to see accept to go away on server");
+      waitForAcceptsInProgressToBe(0);
+
+      // now try it without a close. Server should timeout the mean connect
+      getLogWriter().info("creating mean socket 2");
+      vm0.invoke(createMeanSocket);
+      try {
+        getLogWriter().info("waiting to see it connect on server 2");
+        waitForAcceptsInProgressToBe(1);
+        getLogWriter().info("waiting to see accept to go away on server without us closing");
+        waitForAcceptsInProgressToBe(0);
+      } finally {
+        getLogWriter().info("closing mean socket 2");
+        vm0.invoke(closeMeanSocket);
+      }
+
+//       SerializableRunnable denialOfService = new CacheSerializableRunnable("Do lots of connects") {
+//         public void run2() throws CacheException {
+//           int connectionCount = 0;
+//           ArrayList al = new ArrayList(60000);
+//           try {
+//             InetAddress addr = InetAddress.getLocalHost();
+//             for (;;) {
+//               Socket s = new Socket(addr, port);
+//               al.add(s);
+//               connectionCount++;
+//               getLogWriter().info("connected # " + connectionCount + " s=" + s);
+// //               try {
+// //                 s.close();
+// //               } catch (IOException ignore) {}
+//             }
+//           }
+//           catch (Exception e) {
+//             getLogWriter().info("connected # " + connectionCount
+//                                 + " stopped because of exception " + e);
+//             Iterator it = al.iterator();
+//             while (it.hasNext()) {
+//               Socket s = (Socket)it.next();
+//               try {
+//                 s.close();
+//               } catch (IOException ignore) {}
+//             }
+//           }
+//         }
+//       };
+//       // now pretend to do a denial of service attack by doing a bunch of connects
+//       // really fast and see what that does to the server's fds.
+//       getLogWriter().info("doing denial of service attach");
+//       vm0.invoke(denialOfService);
+//       // @todo darrel: check fd limit?
+    }
+    finally {
+      stopBridgeServers(getCache());
+    }
+    }
+    finally {
+      System.getProperties().remove(AcceptorImpl.ACCEPT_TIMEOUT_PROPERTY_NAME);
+    }
+  }
+
+  public void testSynchronousEvents() throws Exception {
+    InternalClientMembership.setForceSynchronous(true);
+    try {
+      doTestBasicEvents();
+    }
+    finally {
+      InternalClientMembership.setForceSynchronous(false);
+    }
+  }
+  
+  /**
+   * Tests event notification methods on ClientMembership.
+   */
+  public void testBasicEvents() throws Exception {
+    doTestBasicEvents();
+  }
+  
+  public void doTestBasicEvents() throws Exception {
+    final boolean[] fired = new boolean[3];
+    final DistributedMember[] member = new DistributedMember[3];
+    final String[] memberId = new String[3];
+    final boolean[] isClient = new boolean[3];
+    
+    ClientMembershipListener listener = new ClientMembershipListener() {
+      public synchronized void memberJoined(ClientMembershipEvent event) {
+        fired[JOINED] = true;
+        member[JOINED] = event.getMember();
+        memberId[JOINED] = event.getMemberId();
+        isClient[JOINED] = event.isClient();
+        notify();
+      }
+      public synchronized void memberLeft(ClientMembershipEvent event) {
+        fired[LEFT] = true;
+        member[LEFT] = event.getMember();
+        memberId[LEFT] = event.getMemberId();
+        isClient[LEFT] = event.isClient();
+        notify();
+      }
+      public synchronized void memberCrashed(ClientMembershipEvent event) {
+        fired[CRASHED] = true;
+        member[CRASHED] = event.getMember();
+        memberId[CRASHED] = event.getMemberId();
+        isClient[CRASHED] = event.isClient();
+        notify();
+      }
+    };
+    ClientMembership.registerClientMembershipListener(listener);
+    
+    // test JOIN for server
+    DistributedMember serverJoined = new TestDistributedMember("serverJoined");
+    InternalClientMembership.notifyJoined(serverJoined, SERVER);
+    synchronized(listener) {
+      if (!fired[JOINED]) {
+        listener.wait(2000);
+      }
+    }
+    assertTrue(fired[JOINED]);
+    assertEquals(serverJoined, member[JOINED]);
+    assertEquals(serverJoined.getId(), memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    // test JOIN for client
+    DistributedMember clientJoined = new TestDistributedMember("clientJoined");
+    InternalClientMembership.notifyJoined(clientJoined, CLIENT);
+    synchronized(listener) {
+      if (!fired[JOINED]) {
+        listener.wait(2000);
+      }
+    }
+    assertTrue(fired[JOINED]);
+    assertEquals(clientJoined, member[JOINED]);
+    assertEquals(clientJoined.getId(), memberId[JOINED]);
+    assertTrue(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    // test LEFT for server
+    DistributedMember serverLeft = new TestDistributedMember("serverLeft");
+    InternalClientMembership.notifyLeft(serverLeft, SERVER);
+    synchronized(listener) {
+      if (!fired[LEFT]) {
+        listener.wait(2000);
+      }
+    }
+    assertFalse(fired[JOINED]);
+    assertNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertTrue(fired[LEFT]);
+    assertEquals(serverLeft, member[LEFT]);
+    assertEquals(serverLeft.getId(), memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    // test LEFT for client
+    DistributedMember clientLeft = new TestDistributedMember("clientLeft");
+    InternalClientMembership.notifyLeft(clientLeft, CLIENT);
+    synchronized(listener) {
+      if (!fired[LEFT]) {
+        listener.wait(2000);
+      }
+    }
+    assertFalse(fired[JOINED]);
+    assertNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertTrue(fired[LEFT]);
+    assertEquals(clientLeft, member[LEFT]);
+    assertEquals(clientLeft.getId(), memberId[LEFT]);
+    assertTrue(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    // test CRASHED for server
+    DistributedMember serverCrashed = new TestDistributedMember("serverCrashed");
+    InternalClientMembership.notifyCrashed(serverCrashed, SERVER);
+    synchronized(listener) {
+      if (!fired[CRASHED]) {
+        listener.wait(2000);
+      }
+    }
+    assertFalse(fired[JOINED]);
+    assertNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertTrue(fired[CRASHED]);
+    assertEquals(serverCrashed, member[CRASHED]);
+    assertEquals(serverCrashed.getId(), memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    // test CRASHED for client
+    DistributedMember clientCrashed = new TestDistributedMember("clientCrashed");
+    InternalClientMembership.notifyCrashed(clientCrashed, CLIENT);
+    synchronized(listener) {
+      if (!fired[CRASHED]) {
+        listener.wait(2000);
+      }
+    }
+    assertFalse(fired[JOINED]);
+    assertNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertTrue(fired[CRASHED]);
+    assertEquals(clientCrashed, member[CRASHED]);
+    assertEquals(clientCrashed.getId(), memberId[CRASHED]);
+    assertTrue(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+  }
+  
+  /**
+   * Resets all elements of arrays used for listener testing. Boolean values
+   * are reset to false. String values are reset to null.
+   */
+  private void resetArraysForTesting(boolean[] fired, 
+                                     DistributedMember[] member,
+                                     String[] memberId, 
+                                     boolean[] isClient) {
+    for (int i = 0; i < fired.length; i++) {
+      fired[i] = false;
+      member[i] = null;
+      memberId[i] = null;
+      isClient[i] = false;
+    }
+  }
+  
+  /**
+   * Tests unregisterClientMembershipListener to ensure that no further events
+   * are delivered to unregistered listeners.
+   */
+  public void testUnregisterClientMembershipListener() throws Exception {
+    final boolean[] fired = new boolean[1];
+    final DistributedMember[] member = new DistributedMember[1];
+    final String[] memberId = new String[1];
+    final boolean[] isClient = new boolean[1];
+    
+    ClientMembershipListener listener = new ClientMembershipListener() {
+      public synchronized void memberJoined(ClientMembershipEvent event) {
+        fired[0] = true;
+        member[0] = event.getMember();
+        memberId[0] = event.getMemberId();
+        isClient[0] = event.isClient();
+        notify();
+      }
+      public void memberLeft(ClientMembershipEvent event) {
+      }
+      public void memberCrashed(ClientMembershipEvent event) {
+      }
+    };
+    ClientMembership.registerClientMembershipListener(listener);
+    
+    // fire event to make sure listener is registered
+    DistributedMember clientJoined = new TestDistributedMember("clientJoined");
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    synchronized(listener) {
+      if (!fired[0]) {
+        listener.wait(2000);
+      }
+    }
+    assertTrue(fired[0]);
+    assertEquals(clientJoined, member[0]);
+    assertEquals(clientJoined.getId(), memberId[0]);
+    assertTrue(isClient[0]);
+
+    resetArraysForTesting(fired, member, memberId, isClient);
+    assertFalse(fired[0]);
+    assertNull(memberId[0]);
+    assertFalse(isClient[0]);
+
+    // unregister and verify listener is not notified
+    ClientMembership.unregisterClientMembershipListener(listener);
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    synchronized(listener) {
+      listener.wait(20);
+    }
+    assertFalse(fired[0]);
+    assertNull(member[0]);
+    assertNull(memberId[0]);
+    assertFalse(isClient[0]);
+  }
+  
+  public void testMultipleListeners() throws Exception {
+    final int NUM_LISTENERS = 4;
+    final boolean[] fired = new boolean[NUM_LISTENERS];
+    final DistributedMember[] member = new DistributedMember[NUM_LISTENERS];
+    final String[] memberId = new String[NUM_LISTENERS];
+    final boolean[] isClient = new boolean[NUM_LISTENERS];
+    
+    final ClientMembershipListener[] listeners = new ClientMembershipListener[NUM_LISTENERS];
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      final int whichListener = i;
+      listeners[i] = new ClientMembershipListener() {
+        public synchronized void memberJoined(ClientMembershipEvent event) {
+          assertFalse(fired[whichListener]);
+          assertNull(member[whichListener]);
+          assertNull(memberId[whichListener]);
+          assertFalse(isClient[whichListener]);
+          fired[whichListener] = true;
+          member[whichListener] = event.getMember();
+          memberId[whichListener] = event.getMemberId();
+          isClient[whichListener] = event.isClient();
+          notify();
+        }
+        public void memberLeft(ClientMembershipEvent event) {
+        }
+        public void memberCrashed(ClientMembershipEvent event) {
+        }
+      };
+    }
+    
+    final DistributedMember clientJoined = new TestDistributedMember("clientJoined");
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      synchronized(listeners[i]) {
+        listeners[i].wait(20);
+      }
+      assertFalse(fired[i]);
+      assertNull(member[i]);
+      assertNull(memberId[i]);
+      assertFalse(isClient[i]);
+    }
+    
+    // attempt to register same listener twice... 2nd reg should be ignored
+    // failure would cause an assertion failure in memberJoined impl
+    ClientMembership.registerClientMembershipListener(listeners[0]);
+    ClientMembership.registerClientMembershipListener(listeners[0]);
+    
+    ClientMembershipListener[] registeredListeners = 
+      ClientMembership.getClientMembershipListeners();
+    assertEquals(1, registeredListeners.length);
+    assertEquals(listeners[0], registeredListeners[0]);
+    
+    ClientMembership.registerClientMembershipListener(listeners[1]);
+    registeredListeners = ClientMembership.getClientMembershipListeners();
+    assertEquals(2, registeredListeners.length);
+    assertEquals(listeners[0], registeredListeners[0]);
+    assertEquals(listeners[1], registeredListeners[1]);
+
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    synchronized(listeners[1]) {
+      if (!fired[1]) {
+        listeners[1].wait(2000);
+      }
+    }
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      if (i < 2) {
+        assertTrue(fired[i]);
+        assertEquals(clientJoined, member[i]);
+        assertEquals(clientJoined.getId(), memberId[i]);
+        assertTrue(isClient[i]);
+      } else {
+        assertFalse(fired[i]);
+        assertNull(member[i]);
+        assertNull(memberId[i]);
+        assertFalse(isClient[i]);
+      }
+    }
+    resetArraysForTesting(fired, member, memberId, isClient);
+        
+    ClientMembership.unregisterClientMembershipListener(listeners[0]);
+    registeredListeners = ClientMembership.getClientMembershipListeners();
+    assertEquals(1, registeredListeners.length);
+    assertEquals(listeners[1], registeredListeners[0]);
+    
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    synchronized(listeners[1]) {
+      if (!fired[1]) {
+        listeners[1].wait(2000);
+      }
+    }
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      if (i == 1) {
+        assertTrue(fired[i]);
+        assertEquals(clientJoined, member[i]);
+        assertEquals(clientJoined.getId(), memberId[i]);
+        assertTrue(isClient[i]);
+      } else {
+        assertFalse(fired[i]);
+        assertNull(member[i]);
+        assertNull(memberId[i]);
+        assertFalse(isClient[i]);
+      }
+    }
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    ClientMembership.registerClientMembershipListener(listeners[2]);
+    ClientMembership.registerClientMembershipListener(listeners[3]);
+    registeredListeners = ClientMembership.getClientMembershipListeners();
+    assertEquals(3, registeredListeners.length);
+    assertEquals(listeners[1], registeredListeners[0]);
+    assertEquals(listeners[2], registeredListeners[1]);
+    assertEquals(listeners[3], registeredListeners[2]);
+
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    synchronized(listeners[3]) {
+      if (!fired[3]) {
+        listeners[3].wait(2000);
+      }
+    }
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      if (i != 0) {
+        assertTrue(fired[i]);
+        assertEquals(clientJoined, member[i]);
+        assertEquals(clientJoined.getId(), memberId[i]);
+        assertTrue(isClient[i]);
+      } else {
+        assertFalse(fired[i]);
+        assertNull(member[i]);
+        assertNull(memberId[i]);
+        assertFalse(isClient[i]);
+      }
+    }
+    resetArraysForTesting(fired, member, memberId, isClient);
+    
+    ClientMembership.registerClientMembershipListener(listeners[0]);
+    registeredListeners = ClientMembership.getClientMembershipListeners();
+    assertEquals(4, registeredListeners.length);
+    assertEquals(listeners[1], registeredListeners[0]);
+    assertEquals(listeners[2], registeredListeners[1]);
+    assertEquals(listeners[3], registeredListeners[2]);
+    assertEquals(listeners[0], registeredListeners[3]);
+
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    synchronized(listeners[0]) {
+      if (!fired[0]) {
+        listeners[0].wait(2000);
+      }
+    }
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      assertTrue(fired[i]);
+      assertEquals(clientJoined, member[i]);
+      assertEquals(clientJoined.getId(), memberId[i]);
+      assertTrue(isClient[i]);
+    }
+    resetArraysForTesting(fired, member, memberId, isClient);
+    
+    ClientMembership.unregisterClientMembershipListener(listeners[3]);
+    registeredListeners = ClientMembership.getClientMembershipListeners();
+    assertEquals(3, registeredListeners.length);
+    assertEquals(listeners[1], registeredListeners[0]);
+    assertEquals(listeners[2], registeredListeners[1]);
+    assertEquals(listeners[0], registeredListeners[2]);
+    
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    synchronized(listeners[0]) {
+      if (!fired[0]) {
+        listeners[0].wait(2000);
+      }
+    }
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      if (i < 3) {
+        assertTrue(fired[i]);
+        assertEquals(clientJoined, member[i]);
+        assertEquals(clientJoined.getId(), memberId[i]);
+        assertTrue(isClient[i]);
+      } else {
+        assertFalse(fired[i]);
+        assertNull(member[i]);
+        assertNull(memberId[i]);
+        assertFalse(isClient[i]);
+      }
+    }
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    ClientMembership.unregisterClientMembershipListener(listeners[2]);
+    registeredListeners = ClientMembership.getClientMembershipListeners();
+    assertEquals(2, registeredListeners.length);
+    assertEquals(listeners[1], registeredListeners[0]);
+    assertEquals(listeners[0], registeredListeners[1]);
+    
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    synchronized(listeners[0]) {
+      if (!fired[0]) {
+        listeners[0].wait(2000);
+      }
+    }
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      if (i < 2) {
+        assertTrue(fired[i]);
+        assertEquals(clientJoined, member[i]);
+        assertEquals(clientJoined.getId(), memberId[i]);
+        assertTrue(isClient[i]);
+      } else {
+        assertFalse(fired[i]);
+        assertNull(member[i]);
+        assertNull(memberId[i]);
+        assertFalse(isClient[i]);
+      }
+    }
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    ClientMembership.unregisterClientMembershipListener(listeners[1]);
+    ClientMembership.unregisterClientMembershipListener(listeners[0]);
+    registeredListeners = ClientMembership.getClientMembershipListeners();
+    assertEquals(0, registeredListeners.length);
+    
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      synchronized(listeners[i]) {
+        listeners[i].wait(20);
+      }
+      assertFalse(fired[i]);
+      assertNull(member[i]);
+      assertNull(memberId[i]);
+      assertFalse(isClient[i]);
+    }
+    resetArraysForTesting(fired, member, memberId, isClient);
+    
+    ClientMembership.registerClientMembershipListener(listeners[1]);
+    registeredListeners = ClientMembership.getClientMembershipListeners();
+    assertEquals(1, registeredListeners.length);
+    assertEquals(listeners[1], registeredListeners[0]);
+    
+    InternalClientMembership.notifyJoined(clientJoined, true);
+    synchronized(listeners[1]) {
+      if (!fired[1]) {
+        listeners[1].wait(2000);
+      }
+    }
+    for (int i = 0; i < NUM_LISTENERS; i++) {
+      if (i == 1) {
+        assertTrue(fired[i]);
+        assertEquals(clientJoined, member[i]);
+        assertEquals(clientJoined.getId(), memberId[i]);
+        assertTrue(isClient[i]);
+      } else {
+        assertFalse(fired[i]);
+        assertNull(member[i]);
+        assertNull(memberId[i]);
+        assertFalse(isClient[i]);
+      }
+    }
+  }
+ 
+  protected static int testClientMembershipEventsInClient_port;
+  private static int getTestClientMembershipEventsInClient_port() {
+    return testClientMembershipEventsInClient_port;
+  }
+  /**
+   * Tests notification of events in client process. Bridge clients detect
+   * server joins when the client connects to the server. If the server
+   * crashes or departs gracefully, the client will detect this as a crash.
+   */
+  public void testClientMembershipEventsInClient() throws Exception {
+    addExpectedException("IOException");
+    final boolean[] fired = new boolean[3];
+    final DistributedMember[] member = new DistributedMember[3];
+    final String[] memberId = new String[3];
+    final boolean[] isClient = new boolean[3];
+    
+    // create and register ClientMembershipListener in controller vm...
+    ClientMembershipListener listener = new ClientMembershipListener() {
+      public synchronized void memberJoined(ClientMembershipEvent event) {
+        getLogWriter().info("[testClientMembershipEventsInClient] memberJoined: " + event);
+        fired[JOINED] = true;
+        member[JOINED] = event.getMember();
+        memberId[JOINED] = event.getMemberId();
+        isClient[JOINED] = event.isClient();
+        notifyAll();
+      }
+      public synchronized void memberLeft(ClientMembershipEvent event) {
+        getLogWriter().info("[testClientMembershipEventsInClient] memberLeft: " + event);
+//        fail("Please update testClientMembershipEventsInClient to handle memberLeft for BridgeServer.");
+      }
+      public synchronized void memberCrashed(ClientMembershipEvent event) {
+        getLogWriter().info("[testClientMembershipEventsInClient] memberCrashed: " + event);
+        fired[CRASHED] = true;
+        member[CRASHED] = event.getMember();
+        memberId[CRASHED] = event.getMemberId();
+        isClient[CRASHED] = event.isClient();
+        notifyAll();
+      }
+    };
+    ClientMembership.registerClientMembershipListener(listener);
+
+    final VM vm0 = Host.getHost(0).getVM(0);
+    final String name = this.getUniqueName();
+    final int[] ports = new int[1];
+
+    // create BridgeServer in vm0...
+    vm0.invoke(new CacheSerializableRunnable("Create BridgeServer") {
+      public void run2() throws CacheException {
+        try {
+          getLogWriter().info("[testClientMembershipEventsInClient] Create BridgeServer");
+          getSystem();
+          AttributesFactory factory = new AttributesFactory();
+          factory.setScope(Scope.LOCAL);
+          Region region = createRegion(name, factory.create());
+          assertNotNull(region);
+          assertNotNull(getRootRegion().getSubregion(name));
+          testClientMembershipEventsInClient_port = startBridgeServer(0);
+        }
+        catch(IOException e) {
+          getSystem().getLogWriter().fine(new Exception(e));
+          fail("Failed to start CacheServer on VM1: " + e.getMessage());
+        }
+      }
+    });
+    
+    // gather details for later creation of ConnectionPool...
+    ports[0] = vm0.invokeInt(ClientMembershipDUnitTest.class, 
+                             "getTestClientMembershipEventsInClient_port");
+    assertTrue(ports[0] != 0);
+
+    DistributedMember serverMember = (DistributedMember) vm0.invoke(ClientMembershipDUnitTest.class,
+    "getDistributedMember");
+
+    String serverMemberId = (String) vm0.invoke(ClientMembershipDUnitTest.class,
+                                                "getMemberId");
+
+    getLogWriter().info("[testClientMembershipEventsInClient] ports[0]=" + ports[0]);
+    getLogWriter().info("[testClientMembershipEventsInClient] serverMember=" + serverMember);
+    getLogWriter().info("[testClientMembershipEventsInClient] serverMemberId=" + serverMemberId);
+
+    assertFalse(fired[JOINED]);
+    assertNull(member[JOINED]);
+    assertNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(member[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(member[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    
+    // sanity check...
+    getLogWriter().info("[testClientMembershipEventsInClient] sanity check");
+    DistributedMember test = new TestDistributedMember("test");
+    InternalClientMembership.notifyJoined(test, SERVER);
+    synchronized(listener) {
+      if (!fired[JOINED] && !fired[CRASHED]) {
+        listener.wait(2000);
+      }
+    }
+    
+    assertTrue(fired[JOINED]);
+    assertEquals(test, member[JOINED]);
+    assertEquals(test.getId(), memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(member[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(member[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+    
+    // create bridge client in controller vm...
+    getLogWriter().info("[testClientMembershipEventsInClient] create bridge client");
+    Properties config = new Properties();
+    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    getSystem(config);
+    
+    try {
+      getCache();
+      AttributesFactory factory = new AttributesFactory();
+      factory.setScope(Scope.LOCAL);
+      ClientServerTestCase.configureConnectionPool(factory, getServerHostName(Host.getHost(0)), ports, true, -1, -1, null);
+      createRegion(name, factory.create());
+      assertNotNull(getRootRegion().getSubregion(name));
+    }
+    catch (CacheException ex) {
+      fail("While creating Region on Edge", ex);
+    }
+    synchronized(listener) {
+      if (!fired[JOINED] && !fired[CRASHED]) {
+        listener.wait(60 * 1000);
+      }
+    }
+    
+    getLogWriter().info("[testClientMembershipEventsInClient] assert client detected server join");
+    
+    // first check the getCurrentServers() result
+    ClientCache clientCache = (ClientCache)getCache();
+    Set<InetSocketAddress> servers = clientCache.getCurrentServers();
+    assertTrue(!servers.isEmpty());
+    InetSocketAddress serverAddr = servers.iterator().next();
+    InetSocketAddress expectedAddr = new InetSocketAddress(serverMember.getHost(), ports[0]);
+    assertEquals(expectedAddr, serverAddr);
+    
+    // now check listener results
+    assertTrue(fired[JOINED]);
+    assertNotNull(member[JOINED]);
+    assertNotNull(memberId[JOINED]);
+    assertEquals(serverMember, member[JOINED]);
+    assertEquals(serverMemberId, memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(member[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(member[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    vm0.invoke(new SerializableRunnable("Stop BridgeServer") {
+      public void run() {
+        getLogWriter().info("[testClientMembershipEventsInClient] Stop BridgeServer");
+        stopBridgeServers(getCache());
+      }
+    });
+    synchronized(listener) {
+      if (!fired[JOINED] && !fired[CRASHED]) {
+        listener.wait(60 * 1000);
+      }
+    }
+    
+    getLogWriter().info("[testClientMembershipEventsInClient] assert client detected server departure");
+    assertFalse(fired[JOINED]);
+    assertNull(member[JOINED]);
+    assertNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(member[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertTrue(fired[CRASHED]);
+    assertNotNull(member[CRASHED]);
+    assertNotNull(memberId[CRASHED]);
+    assertEquals(serverMember, member[CRASHED]);
+    assertEquals(serverMemberId, memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+    
+    //now test that we redisover the bridge server
+    vm0.invoke(new CacheSerializableRunnable("Recreate BridgeServer") {
+      public void run2() throws CacheException {
+        try {
+          getLogWriter().info("[testClientMembershipEventsInClient] restarting BridgeServer");
+          startBridgeServer(ports[0]);
+        }
+        catch(IOException e) {
+          getSystem().getLogWriter().fine(new Exception(e));
+          fail("Failed to start CacheServer on VM1: " + e.getMessage());
+        }
+      }
+    });
+    synchronized(listener) {
+      if (!fired[JOINED] && !fired[CRASHED]) {
+        listener.wait(60 * 1000);
+      }
+    }
+    
+    getLogWriter().info("[testClientMembershipEventsInClient] assert client detected server recovery");
+    assertTrue(fired[JOINED]);
+    assertNotNull(member[JOINED]);
+    assertNotNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertEquals(serverMember, member[JOINED]);
+    assertEquals(serverMemberId, memberId[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(member[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(member[CRASHED]);
+    assertNull(memberId[CRASHED]);
+  }
+  
+  /**
+   * Tests notification of events in server process. Bridge servers detect
+   * client joins when the client connects to the server.
+   */
+  public void testClientMembershipEventsInServer() throws Exception {
+    final boolean[] fired = new boolean[3];
+    final DistributedMember[] member = new DistributedMember[3];
+    final String[] memberId = new String[3];
+    final boolean[] isClient = new boolean[3];
+    
+    // create and register ClientMembershipListener in controller vm...
+    ClientMembershipListener listener = new ClientMembershipListener() {
+      public synchronized void memberJoined(ClientMembershipEvent event) {
+        getLogWriter().info("[testClientMembershipEventsInServer] memberJoined: " + event);
+        fired[JOINED] = true;
+        member[JOINED] = event.getMember();
+        memberId[JOINED] = event.getMemberId();
+        isClient[JOINED] = event.isClient();
+        notifyAll();
+        assertFalse(fired[LEFT] || fired[CRASHED]);
+      }
+      public synchronized void memberLeft(ClientMembershipEvent event) {
+        getLogWriter().info("[testClientMembershipEventsInServer] memberLeft: " + event);
+        fired[LEFT] = true;
+        member[LEFT] = event.getMember();
+        memberId[LEFT] = event.getMemberId();
+        isClient[LEFT] = event.isClient();
+        notifyAll();
+        assertFalse(fired[JOINED] || fired[CRASHED]);
+      }
+      public synchronized void memberCrashed(ClientMembershipEvent event) {
+        getLogWriter().info("[testClientMembershipEventsInServer] memberCrashed: " + event);
+        fired[CRASHED] = true;
+        member[CRASHED] = event.getMember();
+        memberId[CRASHED] = event.getMemberId();
+        isClient[CRASHED] = event.isClient();
+        notifyAll();
+        assertFalse(fired[JOINED] || fired[LEFT]);
+      }
+    };
+    ClientMembership.registerClientMembershipListener(listener);
+
+    final VM vm0 = Host.getHost(0).getVM(0);
+    final String name = this.getUniqueName();
+    final int[] ports = new int[1];
+
+    // create BridgeServer in controller vm...
+    getLogWriter().info("[testClientMembershipEventsInServer] Create BridgeServer");
+    getSystem();
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    Region region = createRegion(name, factory.create());
+    assertNotNull(region);
+    assertNotNull(getRootRegion().getSubregion(name));
+    
+    ports[0] = startBridgeServer(0);
+    assertTrue(ports[0] != 0);
+    String serverMemberId = getMemberId();
+    DistributedMember serverMember = getDistributedMember();
+
+    getLogWriter().info("[testClientMembershipEventsInServer] ports[0]=" + ports[0]);
+    getLogWriter().info("[testClientMembershipEventsInServer] serverMemberId=" + serverMemberId);
+    getLogWriter().info("[testClientMembershipEventsInServer] serverMember=" + serverMember);
+
+    assertFalse(fired[JOINED]);
+    assertNull(member[JOINED]);
+    assertNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(member[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(member[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    
+    // sanity check...
+    getLogWriter().info("[testClientMembershipEventsInServer] sanity check");
+    DistributedMember test = new TestDistributedMember("test");
+    InternalClientMembership.notifyJoined(test, CLIENT);
+    synchronized(listener) {
+      if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+        listener.wait(2000);
+      }
+    }
+    assertTrue(fired[JOINED]);
+    assertEquals(test, member[JOINED]);
+    assertEquals(test.getId(), memberId[JOINED]);
+    assertTrue(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(member[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(member[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+    
+    final Host host = Host.getHost(0);
+    SerializableRunnable createConnectionPool =
+    new CacheSerializableRunnable("Create connectionPool") {
+      public void run2() throws CacheException {
+        getLogWriter().info("[testClientMembershipEventsInServer] create bridge client");
+        Properties config = new Properties();
+        config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+        getSystem(config);
+        AttributesFactory factory = new AttributesFactory();
+        factory.setScope(Scope.LOCAL);
+        ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, 2, null);
+        createRegion(name, factory.create());
+        assertNotNull(getRootRegion().getSubregion(name));
+      }
+    };
+
+    // create bridge client in vm0...
+    vm0.invoke(createConnectionPool);
+    String clientMemberId = (String) vm0.invoke(ClientMembershipDUnitTest.class,
+                                                "getMemberId");
+    DistributedMember clientMember = (DistributedMember) vm0.invoke(ClientMembershipDUnitTest.class,
+                                                "getDistributedMember");
+                                                
+    synchronized(listener) {
+      if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+        listener.wait(60000);
+      }
+    }
+    
+    getLogWriter().info("[testClientMembershipEventsInServer] assert server detected client join");
+    assertTrue(fired[JOINED]);
+    assertEquals(member[JOINED] + " should equal " + clientMember,
+      clientMember, member[JOINED]);
+    assertEquals(memberId[JOINED] + " should equal " + clientMemberId,
+      clientMemberId, memberId[JOINED]);
+    assertTrue(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(member[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(member[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    pauseForClientToJoin();
+    
+    vm0.invoke(new SerializableRunnable("Stop bridge client") {
+      public void run() {
+        getLogWriter().info("[testClientMembershipEventsInServer] Stop bridge client");
+        getRootRegion().getSubregion(name).close();
+        Map m = PoolManager.getAll();
+        Iterator mit = m.values().iterator();
+        while(mit.hasNext()) {
+          Pool p = (Pool)mit.next();
+          p.destroy();
+        }
+      }
+    });
+
+    synchronized(listener) {
+      if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+        listener.wait(60000);
+      }
+    }
+    
+    getLogWriter().info("[testClientMembershipEventsInServer] assert server detected client left");
+    assertFalse(fired[JOINED]);
+    assertNull(member[JOINED]);
+    assertNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    assertTrue(fired[LEFT]);
+    assertEquals(clientMember, member[LEFT]);
+    assertEquals(clientMemberId, memberId[LEFT]);
+    assertTrue(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(member[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+
+    // reconnect bridge client to test for crashed event
+    vm0.invoke(createConnectionPool);
+    clientMemberId = (String) vm0.invoke(ClientMembershipDUnitTest.class,
+                                         "getMemberId");
+                                                
+    synchronized(listener) {
+      if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+        listener.wait(60000);
+      }
+    }
+    
+    getLogWriter().info("[testClientMembershipEventsInServer] assert server detected client re-join");
+    assertTrue(fired[JOINED]);
+    assertEquals(clientMember, member[JOINED]);
+    assertEquals(clientMemberId, memberId[JOINED]);
+    assertTrue(isClient[JOINED]);
+    assertFalse(fired[LEFT]);
+    assertNull(member[LEFT]);
+    assertNull(memberId[LEFT]);
+    assertFalse(isClient[LEFT]);
+    assertFalse(fired[CRASHED]);
+    assertNull(member[CRASHED]);
+    assertNull(memberId[CRASHED]);
+    assertFalse(isClient[CRASHED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+    
+    pauseForClientToJoin();
+
+    ServerConnection.setForceClientCrashEvent(true);
+    try {
+      vm0.invoke(new SerializableRunnable("Stop bridge client") {
+        public void run() {
+          getLogWriter().info("[testClientMembershipEventsInServer] Stop bridge client");
+          getRootRegion().getSubregion(name).close();
+          Map m = PoolManager.getAll();
+          Iterator mit = m.values().iterator();
+          while(mit.hasNext()) {
+            Pool p = (Pool)mit.next();
+            p.destroy();
+          }
+        }
+      });
+  
+      synchronized(listener) {
+        if (!fired[JOINED] && !fired[LEFT] && !fired[CRASHED]) {
+          listener.wait(60000);
+        }
+      }
+      
+      getLogWriter().info("[testClientMembershipEventsInServer] assert server detected client crashed");
+      assertFalse(fired[JOINED]);
+      assertNull(member[JOINED]);
+      assertNull(memberId[JOINED]);
+      assertFalse(isClient[JOINED]);
+      assertFalse(fired[LEFT]);
+      assertNull(member[LEFT]);
+      assertNull(memberId[LEFT]);
+      assertFalse(isClient[LEFT]);
+      assertTrue(fired[CRASHED]);
+      assertEquals(clientMember, member[CRASHED]);
+      assertEquals(clientMemberId, memberId[CRASHED]);
+      assertTrue(isClient[CRASHED]);
+    }
+    finally {
+      ServerConnection.setForceClientCrashEvent(false);
+    }
+  }
+  
+  /**
+   * The joined event fires when the first client handshake is processed.
+   * This pauses long enough to allow the rest of the client sockets to
+   * complete handshaking before making the client leave. Without doing this
+   * subsequent socket handshakes that are processed could fire join events
+   * after departure events and then a departure event again. If you see
+   * failures in testClientMembershipEventsInServer, try increasing this
+   * timeout.
+   */
+  private void pauseForClientToJoin() {
+    pause(2000);
+  }
+  
+  /** 
+   * Tests registration and event notification in conjunction with 
+   * disconnecting and reconnecting to DistributedSystem. 
+   */
+  public void testLifecycle() throws Exception {
+    final boolean[] fired = new boolean[3];
+    final DistributedMember[] member = new DistributedMember[3];
+    final String[] memberId = new String[3];
+    final boolean[] isClient = new boolean[3];
+    
+    // create and register ClientMembershipListener in controller vm...
+    ClientMembershipListener listener = new ClientMembershipListener() {
+      public synchronized void memberJoined(ClientMembershipEvent event) {
+        assertFalse(fired[JOINED]);
+        assertNull(member[JOINED]);
+        assertNull(memberId[JOINED]);
+        assertFalse(isClient[JOINED]);
+        fired[JOINED] = true;
+        member[JOINED] = event.getMember();
+        memberId[JOINED] = event.getMemberId();
+        isClient[JOINED] = event.isClient();
+        notifyAll();
+      }
+      public synchronized void memberLeft(ClientMembershipEvent event) {
+      }
+      public synchronized void memberCrashed(ClientMembershipEvent event) {
+      }
+    };
+    ClientMembership.registerClientMembershipListener(listener);
+    
+    // create loner in controller vm...
+    Properties config = new Properties();
+    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    getSystem(config);
+    
+    // assert that event is fired while connected
+    DistributedMember serverJoined = new TestDistributedMember("serverJoined");
+    InternalClientMembership.notifyJoined(serverJoined, SERVER);
+    synchronized(listener) {
+      if (!fired[JOINED]) {
+        listener.wait(2000);
+      }
+    }
+    assertTrue(fired[JOINED]);
+    assertEquals(serverJoined, member[JOINED]);
+    assertEquals(serverJoined.getId(), memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+    
+    // assert that event is NOT fired while disconnected
+    disconnectFromDS();
+    
+
+    InternalClientMembership.notifyJoined(serverJoined, SERVER);
+    synchronized(listener) {
+      listener.wait(20);
+    }
+    assertFalse(fired[JOINED]);
+    assertNull(member[JOINED]);
+    assertNull(memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+    resetArraysForTesting(fired, member, memberId, isClient);
+    
+    // assert that event is fired again after reconnecting
+    InternalDistributedSystem sys = getSystem(config);
+    assertTrue(sys.isConnected());
+
+    InternalClientMembership.notifyJoined(serverJoined, SERVER);
+    synchronized(listener) {
+      if (!fired[JOINED]) {
+        listener.wait(2000);
+      }
+    }
+    assertTrue(fired[JOINED]);
+    assertEquals(serverJoined, member[JOINED]);
+    assertEquals(serverJoined.getId(), memberId[JOINED]);
+    assertFalse(isClient[JOINED]);
+  }
+  
+  /**
+   * Starts up server in controller vm and 4 clients, then calls and tests
+   * ClientMembership.getConnectedClients(). 
+   */
+  public void testGetConnectedClients() throws Exception {
+    final String name = this.getUniqueName();
+    final int[] ports = new int[1];
+
+    // create BridgeServer in controller vm...
+    getLogWriter().info("[testGetConnectedClients] Create BridgeServer");
+    getSystem();
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+    Region region = createRegion(name, factory.create());
+    assertNotNull(region);
+    assertNotNull(getRootRegion().getSubregion(name));
+    
+    ports[0] = startBridgeServer(0);
+    assertTrue(ports[0] != 0);
+    String serverMemberId = getMemberId();
+
+    getLogWriter().info("[testGetConnectedClients] ports[0]=" + ports[0]);
+    getLogWriter().info("[testGetConnectedClients] serverMemberId=" + serverMemberId);
+
+    final Host host = Host.getHost(0);
+    SerializableRunnable createPool =
+    new CacheSerializableRunnable("Create connection pool") {
+      public void run2() throws CacheException {
+        getLogWriter().info("[testGetConnectedClients] create bridge client");
+        Properties config = new Properties();
+        config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+        config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+        getSystem(config);
+        AttributesFactory factory = new AttributesFactory();
+        factory.setScope(Scope.LOCAL);
+        ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, -1, null);
+        createRegion(name, factory.create());
+        assertNotNull(getRootRegion().getSubregion(name));
+      }
+    };
+
+    // create bridge client in vm0...
+    final String[] clientMemberIdArray = new String[host.getVMCount()];
+    
+    for (int i = 0; i < host.getVMCount(); i++) { 
+      final VM vm = Host.getHost(0).getVM(i);
+      vm.invoke(createPool);
+      clientMemberIdArray[i] =  String.valueOf(vm.invoke(
+        ClientMembershipDUnitTest.class, "getMemberId"));
+    }
+    Collection clientMemberIds = Arrays.asList(clientMemberIdArray);
+                                                
+    {
+      final int expectedClientCount = clientMemberIds.size();
+      WaitCriterion wc = new WaitCriterion() {
+        public String description() {
+          return "wait for clients";
+        }
+        public boolean done() {
+          Map connectedClients = InternalClientMembership.getConnectedClients(false);
+          if (connectedClients == null) {
+            return false;
+          }
+          if (connectedClients.size() != expectedClientCount) {
+            return false;
+          }
+          return true;
+        }
+      };
+      waitForCriterion(wc, 10000, 100, false);
+    }
+    
+    Map connectedClients = InternalClientMembership.getConnectedClients(false);
+    assertNotNull(connectedClients);
+    assertEquals(clientMemberIds.size(), connectedClients.size());
+    for (Iterator iter = connectedClients.keySet().iterator(); iter.hasNext();) {
+      String connectedClient = (String)iter.next();
+      getLogWriter().info("[testGetConnectedClients] checking for client " + connectedClient);
+      assertTrue(clientMemberIds.contains(connectedClient));
+      getLogWriter().info("[testGetConnectedClients] count for connectedClient: " + 
+                          connectedClients.get(connectedClient));
+    }
+  }
+
+  /**
+   * Starts up 4 server and the controller vm as a client, then calls and tests
+   * ClientMembership.getConnectedServers(). 
+   */
+  public void testGetConnectedServers() throws Exception {
+    final Host host = Host.getHost(0);
+    final String name = this.getUniqueName();
+    final int[] ports = new int[host.getVMCount()];
+    
+    for (int i = 0; i < host.getVMCount(); i++) { 
+      final int whichVM = i;
+      final VM vm = Host.getHost(0).getVM(i);
+      vm.invoke(new CacheSerializableRunnable("Create bridge server") {
+        public void run2() throws CacheException {
+          // create BridgeServer in controller vm...
+          getLogWriter().info("[testGetConnectedServers] Create BridgeServer");
+          getSystem();
+          AttributesFactory factory = new AttributesFactory();
+          factory.setScope(Scope.LOCAL);
+          Region region = createRegion(name+"_"+whichVM, factory.create());
+          assertNotNull(region);
+          assertNotNull(getRootRegion().getSubregion(name+"_"+whichVM));
+          region.put("KEY-1", "VAL-1");
+          
+          try {
+            testGetConnectedServers_port = startBridgeServer(0);
+          }
+          catch (IOException e) {
+            getLogWriter().error("startBridgeServer threw IOException", e);
+            fail("startBridgeServer threw IOException " + e.getMessage());
+          }
+          
+          assertTrue(testGetConnectedServers_port != 0);
+      
+          getLogWriter().info("[testGetConnectedServers] port=" + 
+            ports[whichVM]);
+          getLogWriter().info("[testGetConnectedServers] serverMemberId=" + 
+            getDistributedMember());
+        }
+      });
+      ports[whichVM] = vm.invokeInt(ClientMembershipDUnitTest.class, 
+                                    "getTestGetConnectedServers_port");
+      assertTrue(ports[whichVM] != 0);
+    }
+    
+    getLogWriter().info("[testGetConnectedServers] create bridge client");
+    Properties config = new Properties();
+    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    getSystem(config);
+    getCache();
+    
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+
+    for (int i = 0; i < ports.length; i++) {
+      getLogWriter().info("[testGetConnectedServers] creating connectionpool for " + 
+        getServerHostName(host) + " " + ports[i]);
+      int[] thisServerPorts = new int[] { ports[i] };
+      ClientServerTestCase.configureConnectionPoolWithName(factory, getServerHostName(host), thisServerPorts, false, -1, -1, null,"pooly"+i);
+      Region region = createRegion(name+"_"+i, factory.create());
+      assertNotNull(getRootRegion().getSubregion(name+"_"+i));
+      region.get("KEY-1");
+    }
+
+    {
+      final int expectedVMCount = host.getVMCount();
+      WaitCriterion wc = new WaitCriterion() {
+        public String description() {
+          return "wait for pools and servers";
+        }
+        public boolean done() {
+          if (PoolManager.getAll().size() != expectedVMCount) {
+            return false;
+          }
+          Map connectedServers = InternalClientMembership.getConnectedServers();
+          if (connectedServers == null) {
+            return false;
+          }
+          if (connectedServers.size() != expectedVMCount) {
+            return false;
+          }
+          return true;
+        }
+      };
+      waitForCriterion(wc, 10000, 100, false);
+    }
+
+    {
+      assertEquals(host.getVMCount(), PoolManager.getAll().size());
+      
+    }
+    
+    Map connectedServers = InternalClientMembership.getConnectedServers();
+    assertNotNull(connectedServers);
+    assertEquals(host.getVMCount(), connectedServers.size());
+    for (Iterator iter = connectedServers.keySet().iterator(); iter.hasNext();) {
+      String connectedServer = (String) iter.next();
+      getLogWriter().info("[testGetConnectedServers]  value for connectedServer: " + 
+                          connectedServers.get(connectedServer));
+    }
+  }
+
+  protected static int testGetConnectedServers_port;
+  private static int getTestGetConnectedServers_port() {
+    return testGetConnectedServers_port;
+  }
+
+  /**
+   * Tests getConnectedClients(boolean onlyClientsNotifiedByThisServer) where
+   * onlyClientsNotifiedByThisServer is true.
+   */
+  public void testGetNotifiedClients() throws Exception {
+    final Host host = Host.getHost(0);
+    final String name = this.getUniqueName();
+    final int[] ports = new int[host.getVMCount()];
+    
+    for (int i = 0; i < host.getVMCount(); i++) { 
+      final int whichVM = i;
+      final VM vm = Host.getHost(0).getVM(i);
+      vm.invoke(new CacheSerializableRunnable("Create bridge server") {
+        public void run2() throws CacheException {
+          // create BridgeServer in controller vm...
+          getLogWriter().info("[testGetNotifiedClients] Create BridgeServer");
+          getSystem();
+          AttributesFactory factory = new AttributesFactory();
+          Region region = createRegion(name, factory.create());
+          assertNotNull(region);
+          assertNotNull(getRootRegion().getSubregion(name));
+          region.put("KEY-1", "VAL-1");
+          
+          try {
+            testGetNotifiedClients_port = startBridgeServer(0);
+          }
+          catch (IOException e) {
+            getLogWriter().error("startBridgeServer threw IOException", e);
+            fail("startBridgeServer threw IOException " + e.getMessage());
+          }
+          
+          assertTrue(testGetNotifiedClients_port != 0);
+      
+          getLogWriter().info("[testGetNotifiedClients] port=" + 
+            ports[whichVM]);
+          getLogWriter().info("[testGetNotifiedClients] serverMemberId=" + 
+            getMemberId());
+        }
+      });
+      ports[whichVM] = vm.invokeInt(ClientMembershipDUnitTest.class, 
+                                    "getTestGetNotifiedClients_port");
+      assertTrue(ports[whichVM] != 0);
+    }
+    
+    getLogWriter().info("[testGetNotifiedClients] create bridge client");
+    Properties config = new Properties();
+    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    getSystem(config);
+    getCache();
+    
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+
+    getLogWriter().info("[testGetNotifiedClients] creating connection pool");
+    ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, true, -1, -1, null);
+    Region region = createRegion(name, factory.create());
+    assertNotNull(getRootRegion().getSubregion(name));
+    region.registerInterest("KEY-1");
+    region.get("KEY-1");
+
+    final String clientMemberId = getMemberId();
+    
+    pauseForClientToJoin();
+    
+    // assertions go here
+    int[] clientCounts = new int[host.getVMCount()];
+    
+    // only one server vm will have that client for updating
+    for (int i = 0; i < host.getVMCount(); i++) { 
+      final int whichVM = i;
+      final VM vm = Host.getHost(0).getVM(i);
+      vm.invoke(new CacheSerializableRunnable("Create bridge server") {
+        public void run2() throws CacheException {
+          Map clients = InternalClientMembership.getConnectedClients(true);
+          assertNotNull(clients);
+          testGetNotifiedClients_clientCount = clients.size();
+          if (testGetNotifiedClients_clientCount > 0) {
+            // assert that the clientMemberId matches
+            assertEquals(clientMemberId, clients.keySet().iterator().next());
+          }
+        }
+      });
+      clientCounts[whichVM] = vm.invokeInt(ClientMembershipDUnitTest.class, 
+                              "getTestGetNotifiedClients_clientCount");
+    }
+    
+    // only one server should have a notifier for this client...
+    int totalClientCounts = 0;
+    for (int i = 0; i < clientCounts.length; i++) {
+      totalClientCounts += clientCounts[i];
+    }
+    // this assertion fails because the count is 4
+    //assertEquals(1, totalClientCounts);
+  }
+  protected static int testGetNotifiedClients_port;
+  private static int getTestGetNotifiedClients_port() {
+    return testGetNotifiedClients_port;
+  }
+  protected static int testGetNotifiedClients_clientCount;
+  private static int getTestGetNotifiedClients_clientCount() {
+    return testGetNotifiedClients_clientCount;
+  }
+
+  // Simple DistributedMember implementation
+  static final class TestDistributedMember implements DistributedMember {
+    
+    private String host;
+    
+    public TestDistributedMember(String host) {
+      this.host = host;
+    }
+
+    public String getName() {
+      return "";
+    }
+
+    public String getHost() {
+      return this.host;
+    }
+
+    public Set getRoles() {
+      return new HashSet();
+    }
+
+    public int getProcessId() {
+      return 0;
+    }
+
+    public String getId() {
+      return this.host;
+    }
+    
+    public int compareTo(DistributedMember o) {
+      if ((o == null) || !(o instanceof TestDistributedMember)) {
+        throw new InternalGemFireException("Invalidly comparing TestDistributedMember to " + o);
+      }
+      
+      TestDistributedMember tds = (TestDistributedMember) o;
+      return getHost().compareTo(tds.getHost());
+    }
+    
+    public boolean equals(Object obj) {
+      if ((obj == null) || !(obj instanceof TestDistributedMember)) {
+        return false;
+      }
+      return compareTo((TestDistributedMember)obj) == 0;
+    }
+    
+    public int hashCode() {
+      return getHost().hashCode();
+    }
+    
+    public DurableClientAttributes getDurableClientAttributes() {
+      
+      return null;
+    }
+
+    public List<String> getGroups() {
+      return Collections.emptyList();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipSelectorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipSelectorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipSelectorDUnitTest.java
new file mode 100644
index 0000000..ebf894e
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientMembershipSelectorDUnitTest.java
@@ -0,0 +1,16 @@
+package com.gemstone.gemfire.cache30;
+
+/**
+ * Same as parent but uses selector in server
+ *
+ * @author darrel
+ * @since 5.1
+ */
+public class ClientMembershipSelectorDUnitTest extends ClientMembershipDUnitTest {
+  public ClientMembershipSelectorDUnitTest(String name) {
+    super(name);
+  }
+  protected int getMaxThreads() {
+    return 2;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientRegisterInterestDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientRegisterInterestDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientRegisterInterestDUnitTest.java
new file mode 100644
index 0000000..b49ec56
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientRegisterInterestDUnitTest.java
@@ -0,0 +1,418 @@
+/*=========================================================================
+ * 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.Properties;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.client.internal.PoolImpl;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.cache.client.SubscriptionNotEnabledException;
+
+import dunit.DistributedTestCase;
+import dunit.Host;
+import dunit.VM;
+
+/**
+ * Tests the client register interest
+ *
+ * @author Kirk Lund
+ * @since 4.2.3
+ */
+public class ClientRegisterInterestDUnitTest extends ClientServerTestCase {
+
+  public ClientRegisterInterestDUnitTest(String name) {
+    super(name);
+  }
+  
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+    disconnectAllFromDS(); // cleans up bridge server and client and lonerDS
+  }
+  
+  /**
+   * Tests for Bug 35381 Calling register interest if 
+   * establishCallbackConnection is not set causes bridge server NPE.
+   */
+  public void testBug35381() throws Exception {
+    final Host host = Host.getHost(0);
+    final String name = this.getUniqueName();
+    final int[] ports = new int[1]; // 1 server in this test
+    
+    final int whichVM = 0;
+    final VM vm = Host.getHost(0).getVM(whichVM);
+    vm.invoke(new CacheSerializableRunnable("Create bridge server") {
+      public void run2() throws CacheException {
+        getLogWriter().info("[testBug35381] Create BridgeServer");
+        getSystem();
+        AttributesFactory factory = new AttributesFactory();
+        factory.setScope(Scope.LOCAL);
+        Region region = createRegion(name, factory.create());
+        assertNotNull(region);
+        assertNotNull(getRootRegion().getSubregion(name));
+        region.put("KEY-1", "VAL-1");
+        
+        try {
+          bridgeServerPort = startBridgeServer(0);
+        }
+        catch (IOException e) {
+          getLogWriter().error("startBridgeServer threw IOException", e);
+          fail("startBridgeServer threw IOException " + e.getMessage());
+        }
+        
+        assertTrue(bridgeServerPort != 0);
+    
+        getLogWriter().info("[testBug35381] port=" + bridgeServerPort);
+        getLogWriter().info("[testBug35381] serverMemberId=" + getMemberId());
+      }
+    });
+    ports[whichVM] = vm.invokeInt(ClientRegisterInterestDUnitTest.class, 
+                                  "getBridgeServerPort");
+    assertTrue(ports[whichVM] != 0);
+    
+    getLogWriter().info("[testBug35381] create bridge client");
+    Properties config = new Properties();
+    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    getSystem(config);
+    getCache();
+    
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+
+    getLogWriter().info("[testBug35381] creating connection pool");
+    boolean establishCallbackConnection = false; // SOURCE OF BUG 35381
+    ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, establishCallbackConnection, -1, -1, null);
+    Region region = createRegion(name, factory.create());
+    assertNotNull(getRootRegion().getSubregion(name));
+    try {
+      region.registerInterest("KEY-1");
+      fail("registerInterest failed to throw SubscriptionNotEnabledException with establishCallbackConnection set to false"); 
+    }
+    catch (SubscriptionNotEnabledException expected) {
+    }
+  }
+  protected static int bridgeServerPort;
+  private static int getBridgeServerPort() {
+    return bridgeServerPort;
+  }
+  
+  /**
+   * Tests failover of register interest from client point of view. Related
+   * bugs include:
+   *
+   * <p>Bug 35654 "failed re-registration may never be detected and thus
+   * may never re-re-register"
+   *
+   * <p>Bug 35639 "registerInterest re-registration happens everytime a healthy
+   * server is detected"
+   *
+   * <p>Bug 35655 "a single failed re-registration causes all other pending
+   * re-registrations to be cancelled"
+   */
+  public void _testRegisterInterestFailover() throws Exception {
+    // controller is bridge client
+    
+    final Host host = Host.getHost(0);
+    final String name = this.getUniqueName();
+    final String regionName1 = name+"-1";
+    final String regionName2 = name+"-2";
+    final String regionName3 = name+"-3";
+    final String key1 = "KEY-"+regionName1+"-1";
+    final String key2 = "KEY-"+regionName1+"-2";
+    final String key3 = "KEY-"+regionName1+"-3";
+    final int[] ports = new int[3]; // 3 servers in this test
+    
+    // create first bridge server with region for client...
+    final int firstServerIdx = 0;
+    final VM firstServerVM = Host.getHost(0).getVM(firstServerIdx);
+    firstServerVM.invoke(new CacheSerializableRunnable("Create first bridge server") {
+      public void run2() throws CacheException {
+        getLogWriter().info("[testRegisterInterestFailover] Create first bridge server");
+        getSystem();
+        AttributesFactory factory = new AttributesFactory();
+        factory.setScope(Scope.LOCAL);
+        Region region1 = createRootRegion(regionName1, factory.create());
+        Region region2 = createRootRegion(regionName2, factory.create());
+        Region region3 = createRootRegion(regionName3, factory.create());
+        region1.put(key1, "VAL-1");
+        region2.put(key2, "VAL-1");
+        region3.put(key3, "VAL-1");
+        
+        try {
+          bridgeServerPort = startBridgeServer(0);
+        }
+        catch (IOException e) {
+          getLogWriter().error("startBridgeServer threw IOException", e);
+          fail("startBridgeServer threw IOException " + e.getMessage());
+        }
+        
+        assertTrue(bridgeServerPort != 0);
+    
+        getLogWriter().info("[testRegisterInterestFailover] " +
+          "firstServer port=" + bridgeServerPort);
+        getLogWriter().info("[testRegisterInterestFailover] " +
+          "firstServer memberId=" + getMemberId());
+      }
+    });
+
+    // create second bridge server missing region for client...
+    final int secondServerIdx = 1;
+    final VM secondServerVM = Host.getHost(0).getVM(secondServerIdx);
+    secondServerVM.invoke(new CacheSerializableRunnable("Create second bridge server") {
+      public void run2() throws CacheException {
+        getLogWriter().info("[testRegisterInterestFailover] Create second bridge server");
+        getSystem();
+        AttributesFactory factory = new AttributesFactory();
+        factory.setScope(Scope.LOCAL);
+        Region region1 = createRootRegion(regionName1, factory.create());
+        Region region3 = createRootRegion(regionName3, factory.create());
+        region1.put(key1, "VAL-2");
+        region3.put(key3, "VAL-2");
+        
+        try {
+          bridgeServerPort = startBridgeServer(0);
+        }
+        catch (IOException e) {
+          getLogWriter().error("startBridgeServer threw IOException", e);
+          fail("startBridgeServer threw IOException " + e.getMessage());
+        }
+        
+        assertTrue(bridgeServerPort != 0);
+    
+        getLogWriter().info("[testRegisterInterestFailover] " +
+          "secondServer port=" + bridgeServerPort);
+        getLogWriter().info("[testRegisterInterestFailover] " +
+          "secondServer memberId=" + getMemberId());
+      }
+    });
+
+    // get the bridge server ports...
+    ports[firstServerIdx] = firstServerVM.invokeInt(
+      ClientRegisterInterestDUnitTest.class, "getBridgeServerPort");
+    assertTrue(ports[firstServerIdx] != 0);
+    ports[secondServerIdx] = secondServerVM.invokeInt(
+      ClientRegisterInterestDUnitTest.class, "getBridgeServerPort");
+    assertTrue(ports[secondServerIdx] != 0);
+    assertTrue(ports[firstServerIdx] != ports[secondServerIdx]);
+    
+    // stop second and third servers
+    secondServerVM.invoke(new CacheSerializableRunnable("Stop second bridge server") {
+      public void run2() throws CacheException {
+        stopBridgeServers(getCache());
+      }
+    });
+    
+    // create the bridge client
+    getLogWriter().info("[testBug35654] create bridge client");
+    Properties config = new Properties();
+    config.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    config.setProperty(DistributionConfig.LOCATORS_NAME, "");
+    getSystem(config);
+    getCache();
+    
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.LOCAL);
+
+    getLogWriter().info("[testRegisterInterestFailover] creating connection pool");
+    boolean establishCallbackConnection = true;
+    final PoolImpl p = (PoolImpl)ClientServerTestCase.configureConnectionPool(factory, getServerHostName(host), ports, establishCallbackConnection, -1, -1, null);
+
+    final Region region1 = createRootRegion(regionName1, factory.create());
+    final Region region2 = createRootRegion(regionName2, factory.create());
+    final Region region3 = createRootRegion(regionName3, factory.create());
+
+    assertTrue(region1.getInterestList().isEmpty());
+    assertTrue(region2.getInterestList().isEmpty());
+    assertTrue(region3.getInterestList().isEmpty());
+
+    region1.registerInterest(key1);
+    region2.registerInterest(key2);
+    region3.registerInterest(key3);
+
+    assertTrue(region1.getInterestList().contains(key1));
+    assertTrue(region2.getInterestList().contains(key2));
+    assertTrue(region3.getInterestList().contains(key3));
+    
+    assertTrue(region1.getInterestListRegex().isEmpty());
+    assertTrue(region2.getInterestListRegex().isEmpty());
+    assertTrue(region3.getInterestListRegex().isEmpty());
+    
+    // get ConnectionProxy and wait until connected to first server
+    WaitCriterion ev = new WaitCriterion() {
+      public boolean done() {
+        return p.getPrimaryPort() != -1;
+      }
+      public String description() {
+        return "primary port remained invalid";
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 10 * 1000, 200, true);
+    assertEquals(ports[firstServerIdx], p.getPrimaryPort()); 
+    
+    // assert intial values
+    assertEquals("VAL-1", region1.get(key1));
+    assertEquals("VAL-1", region2.get(key2));
+    assertEquals("VAL-1", region3.get(key3));
+    
+    // do puts on server1 and make sure values come thru for all 3 registrations
+    firstServerVM.invoke(new CacheSerializableRunnable("Puts from first bridge server") {
+      public void run2() throws CacheException {
+        Region region1 = getCache().getRegion(regionName1);
+        region1.put(key1, "VAL-1-1");
+        Region region2 = getCache().getRegion(regionName2);
+        region2.put(key2, "VAL-1-1");
+        Region region3 = getCache().getRegion(regionName3);
+        region3.put(key3, "VAL-1-1");
+      }
+    });
+
+    ev = new WaitCriterion() {
+      public boolean done() {
+        if (!"VAL-1-1".equals(region1.get(key1)) || 
+            !"VAL-1-1".equals(region2.get(key2)) ||
+            !"VAL-1-1".equals(region3.get(key3))
+            ) return  false;
+        return true;
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 10 * 1000, 200, true);
+    assertEquals("VAL-1-1", region1.get(key1));
+    assertEquals("VAL-1-1", region2.get(key2));
+    assertEquals("VAL-1-1", region3.get(key3));
+    
+    // force failover to server 2
+    secondServerVM.invoke(new CacheSerializableRunnable("Start second bridge server") {
+      public void run2() throws CacheException {
+        try {
+          startBridgeServer(ports[secondServerIdx]);
+        }
+        catch (IOException e) {
+          getLogWriter().error("startBridgeServer threw IOException", e);
+          fail("startBridgeServer threw IOException " + e.getMessage());
+        }
+      }
+    });
+   
+    firstServerVM.invoke(new CacheSerializableRunnable("Stop first bridge server") {
+      public void run2() throws CacheException {
+        stopBridgeServers(getCache());
+      }
+    });
+
+    // wait for failover to second server
+    ev = new WaitCriterion() {
+      public boolean done() {
+        return ports[secondServerIdx] == p.getPrimaryPort();
+      }
+      public String description() {
+        return "primary port never became " + ports[secondServerIdx];
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 100 * 1000, 200, true);
+    
+    try {
+      assertEquals(null, region2.get(key2));
+      fail("CacheLoaderException expected");
+    }
+    catch (com.gemstone.gemfire.cache.CacheLoaderException e) {
+    }
+  
+    // region2 registration should be gone now
+    // do puts on server2 and make sure values come thru for only 2 registrations
+    secondServerVM.invoke(new CacheSerializableRunnable("Puts from second bridge server") {
+      public void run2() throws CacheException {
+        AttributesFactory factory = new AttributesFactory();
+        factory.setScope(Scope.LOCAL);
+        createRootRegion(regionName2, factory.create());
+      }
+    });
+    
+    // assert that there is no actively registered interest on region2
+    assertTrue(region2.getInterestList().isEmpty());
+    assertTrue(region2.getInterestListRegex().isEmpty());
+
+    region2.put(key2, "VAL-0");
+    
+    secondServerVM.invoke(new CacheSerializableRunnable("Put from second bridge server") {
+      public void run2() throws CacheException {
+        Region region1 = getCache().getRegion(regionName1);
+        region1.put(key1, "VAL-2-2");
+        Region region2 = getCache().getRegion(regionName2);
+        region2.put(key2, "VAL-2-1");
+        Region region3 = getCache().getRegion(regionName3);
+        region3.put(key3, "VAL-2-2");
+      }
+    });
+    
+    // wait for updates to come thru
+    ev = new WaitCriterion() {
+      public boolean done() {
+        if (!"VAL-2-2".equals(region1.get(key1)) || 
+            !"VAL-2-2".equals(region3.get(key3)))
+          return false;
+        return true;
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 100 * 1000, 200, true);
+    assertEquals("VAL-2-2", region1.get(key1));
+    assertEquals("VAL-0",   region2.get(key2));
+    assertEquals("VAL-2-2", region3.get(key3));
+
+    // assert again that there is no actively registered interest on region2
+    assertTrue(region2.getInterestList().isEmpty());
+
+    // register interest again on region2 and make
+    region2.registerInterest(key2);
+    assertEquals("VAL-2-1", region2.get(key2));
+    
+    secondServerVM.invoke(new CacheSerializableRunnable("Put from second bridge server") {
+      public void run2() throws CacheException {
+        Region region1 = getCache().getRegion(regionName1);
+        region1.put(key1, "VAL-2-3");
+        Region region2 = getCache().getRegion(regionName2);
+        region2.put(key2, "VAL-2-2");
+        Region region3 = getCache().getRegion(regionName3);
+        region3.put(key3, "VAL-2-3");
+      }
+    });
+    
+    // wait for updates to come thru
+    ev = new WaitCriterion() {
+      public boolean done() {
+        if (!"VAL-2-3".equals(region1.get(key1)) || 
+            !"VAL-2-2".equals(region2.get(key2)) ||
+            !"VAL-2-3".equals(region3.get(key3)))
+          return false;
+        return true;
+      }
+      public String description() {
+        return null;
+      }
+    };
+    DistributedTestCase.waitForCriterion(ev, 100 * 1000, 200, true);
+    assertEquals("VAL-2-3", region1.get(key1));
+    assertEquals("VAL-2-2", region2.get(key2));
+    assertEquals("VAL-2-3", region3.get(key3));
+
+    // assert public methods report actively registered interest on region2
+    assertTrue(region2.getInterestList().contains(key2));
+  }
+  
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2eb4e175/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientRegisterInterestSelectorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientRegisterInterestSelectorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientRegisterInterestSelectorDUnitTest.java
new file mode 100644
index 0000000..86c12f1
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/ClientRegisterInterestSelectorDUnitTest.java
@@ -0,0 +1,16 @@
+package com.gemstone.gemfire.cache30;
+
+/**
+ * Same as parent but uses selector in server
+ *
+ * @author darrel
+ * @since 5.1
+ */
+public class ClientRegisterInterestSelectorDUnitTest extends ClientRegisterInterestDUnitTest {
+  public ClientRegisterInterestSelectorDUnitTest(String name) {
+    super(name);
+  }
+  protected int getMaxThreads() {
+    return 2;
+  }
+}