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

[01/50] [abbrv] incubator-geode git commit: GEODE-655 Website sentence needs improvement

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-14 4b56f5e45 -> 71e1b1fff


GEODE-655 Website sentence needs improvement

The first sentence intends the opposite of what it says. Thus, it
should be changed.


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

Branch: refs/heads/feature/GEODE-14
Commit: f86ddd4dc32a78a84d186cbc01fa3dccce060340
Parents: 34eb0fe
Author: Karen Miller <km...@pivotal.io>
Authored: Fri Dec 11 09:30:35 2015 -0800
Committer: Karen Miller <km...@pivotal.io>
Committed: Fri Dec 11 09:30:35 2015 -0800

----------------------------------------------------------------------
 gemfire-site/website/content/index.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f86ddd4d/gemfire-site/website/content/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/index.html b/gemfire-site/website/content/index.html
index 5e6a474..b0ef7ad 100644
--- a/gemfire-site/website/content/index.html
+++ b/gemfire-site/website/content/index.html
@@ -8,7 +8,7 @@ title: Performance is key. Consistency is a must.
             <img class="logo-title img-responsive hidden-xs" src="img/apache_geode_logo.png" />
             <div class="text-container">
                 <h2 class="tagline"><em>Performance</em> is key. <em>Consistency</em> is a must.</h2>
-                <p class="description">Solving low latency data management problems at very high concurrency since 2002.<br/>
+                <p class="description">Providing low latency, high concurrency data management solutions since 2002.<br/>
                   <br/>Build high-speed, data-intensive applications that elastically meet performance requirements at any scale.<br/>
                   Take advantage of Apache Geode's unique technology that blends advanced techniques for data replication, partitioning and distributed processing.
 


[07/50] [abbrv] incubator-geode git commit: Removing TCPConduit's Stub ID class

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
index a2801c1..41e5837 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
@@ -45,6 +45,7 @@ import org.apache.logging.log4j.Logger;
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
@@ -63,7 +64,7 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 
 /** <p>TCPConduit manages a server socket and a collection of connections to
-    other systems.  Connections are identified by host/port Stubs.
+    other systems.  Connections are identified by DistributedMember IDs.
     These types of messages are currently supported:</p><pre>
 
        DistributionMessage - message is delivered to the server's
@@ -175,9 +176,8 @@ public class TCPConduit implements Runnable {
   
   ////////////////// runtime state that is re-initialized on a restart
 
-  /** id is an endpoint Stub representing this server.  It holds the
-      actual port the server is listening on */
-  private Stub id;
+  /** server socket address */
+  private InetSocketAddress id;
 
   protected volatile boolean stopped;
 
@@ -351,7 +351,7 @@ public class TCPConduit implements Runnable {
     try {
       localPort = socket.getLocalPort();
 
-      id = new Stub(socket.getInetAddress(), localPort, 0);
+      id = new InetSocketAddress(socket.getInetAddress(), localPort);
       stopped = false;
       ThreadGroup group =
         LoggingThreadGroup.createThreadGroup("P2P Listener Threads", logger);
@@ -371,23 +371,12 @@ public class TCPConduit implements Runnable {
       }
     }
     catch (IOException io) {
-      String s = "While creating ServerSocket and Stub on port " + p;
+      String s = "While creating ServerSocket on port " + p;
       throw new ConnectionException(s, io);
     }
     this.port = localPort;
   }
   
-  /**
-   * After startup we install the view ID into the conduit stub to avoid
-   * confusion during overlapping shutdown/startup from the same member.
-   * 
-   * @param viewID
-   */
-  public void setVmViewID(int viewID) {
-    this.id.setViewID(viewID);
-  }
-
-
   /** creates the server sockets.  This can be used to recreate the
    *  socket using this.port and this.bindAddress, which must be set
    *  before invoking this method.
@@ -596,7 +585,7 @@ public class TCPConduit implements Runnable {
   public void run() {
     ConnectionTable.threadWantsSharedResources();
     if (logger.isTraceEnabled(LogMarker.DM)) {
-      logger.trace(LogMarker.DM, "Starting P2P Listener on  {}", this.getId());
+      logger.trace(LogMarker.DM, "Starting P2P Listener on  {}", id);
     }
     for(;;) {
       SystemFailure.checkFailure();
@@ -731,7 +720,7 @@ public class TCPConduit implements Runnable {
     } // for
 
     if (logger.isTraceEnabled(LogMarker.DM)) {
-      logger.debug("Stopped P2P Listener on  {}", this.getId());
+      logger.debug("Stopped P2P Listener on  {}", id);
     }
   }
 
@@ -807,7 +796,7 @@ public class TCPConduit implements Runnable {
    * @since 5.1
    */
   public void getThreadOwnedOrderedConnectionState(
-    Stub member,
+    DistributedMember member,
     Map result)
   {
     getConTable().getThreadOwnedOrderedConnectionState(member, result);
@@ -819,7 +808,7 @@ public class TCPConduit implements Runnable {
    * with the key
    * @since 5.1
    */
-  public void waitForThreadOwnedOrderedConnectionState(Stub member, Map channelState)
+  public void waitForThreadOwnedOrderedConnectionState(DistributedMember member, Map channelState)
     throws InterruptedException
   {
     // if (Thread.interrupted()) throw new InterruptedException(); not necessary done in waitForThreadOwnedOrderedConnectionState
@@ -842,13 +831,12 @@ public class TCPConduit implements Runnable {
       msg.setBytesRead(bytesRead);
       msg.setSender(receiver.getRemoteAddress());
       msg.setSharedReceiver(receiver.isSharedResource());
-      directChannel.receive(msg, bytesRead, receiver.getRemoteId());
+      directChannel.receive(msg, bytesRead);
     }
   }
 
-  /** gets the Stub representing this conduit's ServerSocket endpoint.  This
-      is used to generate other stubs containing endpoint information. */
-  public Stub getId() {
+  /** gets the address of this conduit's ServerSocket endpoint */
+  public InetSocketAddress getId() {
     return id;
   }
 
@@ -870,21 +858,16 @@ public class TCPConduit implements Runnable {
   }
 
   
-  /** gets the channel that is used to process non-Stub messages */
+  /** gets the channel that is used to process non-DistributedMember messages */
   public DirectChannel getDirectChannel() {
     return directChannel;
   }
 
-  public InternalDistributedMember getMemberForStub(Stub s, boolean validate) {
-    return membershipManager.getMemberForStub(s, validate);
-  }
-  
   public void setLocalAddr(InternalDistributedMember addr) {
     localAddr = addr;
-    this.id.setViewID(addr.getVmViewId());
   }
   
-  public InternalDistributedMember getLocalId() {
+  public InternalDistributedMember getLocalAddr() {
     return localAddr;
   }
 
@@ -894,7 +877,6 @@ public class TCPConduit implements Runnable {
    * member is in the membership view and the system is not shutting down.
    * 
    * @param memberAddress the IDS associated with the remoteId
-   * @param remoteId the TCPConduit stub for this member
    * @param preserveOrder whether this is an ordered or unordered connection
    * @param retry false if this is the first attempt
    * @param startTime the time this operation started
@@ -902,8 +884,8 @@ public class TCPConduit implements Runnable {
    * @param ackSATimeout the ack-severe-alert-threshold * 1000 for the operation to be transmitted (or zero)
    * @return the connection
    */
-  public Connection getConnection(InternalDistributedMember memberAddress, Stub remoteId, final boolean preserveOrder, boolean retry,
-      long startTime, long ackTimeout, long ackSATimeout)
+  public Connection getConnection(InternalDistributedMember memberAddress, final boolean preserveOrder, boolean retry, long startTime,
+      long ackTimeout, long ackSATimeout)
     throws java.io.IOException, DistributedSystemDisconnectedException
   {
     //final boolean preserveOrder = (processorType == DistributionManager.SERIAL_EXECUTOR )|| (processorType == DistributionManager.PARTITIONED_REGION_EXECUTOR);
@@ -922,11 +904,7 @@ public class TCPConduit implements Runnable {
       // problems.  Tear down the connection so that it gets
       // rebuilt.
       if (retry || conn != null) { // not first time in loop
-        // Consult with the membership manager; if member has gone away,
-        // there will not be an entry for this stub.
-        InternalDistributedMember m = this.membershipManager.getMemberForStub(remoteId, true);
-        if (m == null) {
-          // OK, the member left.  Just register an error.
+        if (!membershipManager.memberExists(memberAddress) || membershipManager.isShunned(memberAddress) || membershipManager.shutdownInProgress()) {
           throw new IOException(LocalizedStrings.TCPConduit_TCPIP_CONNECTION_LOST_AND_MEMBER_IS_NOT_IN_VIEW.toLocalizedString());
         }
         // bug35953: Member is still in view; we MUST NOT give up!
@@ -941,15 +919,14 @@ public class TCPConduit implements Runnable {
         }
         
         // try again after sleep
-        m = this.membershipManager.getMemberForStub(remoteId, true);
-        if (m == null) {
+        if (!membershipManager.memberExists(memberAddress) || membershipManager.isShunned(memberAddress)) {
           // OK, the member left.  Just register an error.
           throw new IOException(LocalizedStrings.TCPConduit_TCPIP_CONNECTION_LOST_AND_MEMBER_IS_NOT_IN_VIEW.toLocalizedString());
         }
         
         // Print a warning (once)
         if (memberInTrouble == null) {
-          memberInTrouble = m;
+          memberInTrouble = memberAddress;
           logger.warn(LocalizedMessage.create(LocalizedStrings.TCPConduit_ATTEMPTING_TCPIP_RECONNECT_TO__0, memberInTrouble));
         }
         else {
@@ -963,8 +940,8 @@ public class TCPConduit implements Runnable {
         if (conn != null) {
           try { 
             if (logger.isDebugEnabled()) {
-              logger.debug("Closing old connection.  conn={} before retrying. remoteID={} memberInTrouble={}",
-                  conn, remoteId,  memberInTrouble);
+              logger.debug("Closing old connection.  conn={} before retrying. memberInTrouble={}",
+                  conn, memberInTrouble);
             }
             conn.closeForReconnect("closing before retrying"); 
           } 
@@ -985,10 +962,10 @@ public class TCPConduit implements Runnable {
         boolean debugRetry = false;
         do {
           retryForOldConnection = false;
-          conn = getConTable().get(remoteId, preserveOrder, startTime, ackTimeout, ackSATimeout);
+          conn = getConTable().get(memberAddress, preserveOrder, startTime, ackTimeout, ackSATimeout);
           if (conn == null) {
             // conduit may be closed - otherwise an ioexception would be thrown
-            problem = new IOException(LocalizedStrings.TCPConduit_UNABLE_TO_RECONNECT_TO_SERVER_POSSIBLE_SHUTDOWN_0.toLocalizedString(remoteId));
+            problem = new IOException(LocalizedStrings.TCPConduit_UNABLE_TO_RECONNECT_TO_SERVER_POSSIBLE_SHUTDOWN_0.toLocalizedString(memberAddress));
           } else if (conn.isClosing() || !conn.getRemoteAddress().equals(memberAddress)) {
             if (logger.isDebugEnabled()) {
               logger.debug("Got an old connection for {}: {}@{}", memberAddress, conn, conn.hashCode());
@@ -1027,15 +1004,14 @@ public class TCPConduit implements Runnable {
 
       if (problem != null) {
         // Some problems are not recoverable; check and error out early.
-        InternalDistributedMember m = this.membershipManager.getMemberForStub(remoteId, true);
-        if (m == null) { // left the view
+        if (!membershipManager.memberExists(memberAddress) || membershipManager.isShunned(memberAddress)) { // left the view
           // Bracket our original warning
           if (memberInTrouble != null) {
             // make this msg info to bracket warning
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.TCPConduit_ENDING_RECONNECT_ATTEMPT_BECAUSE_0_HAS_DISAPPEARED, memberInTrouble));
           }
-          throw new IOException(LocalizedStrings.TCPConduit_PEER_HAS_DISAPPEARED_FROM_VIEW.toLocalizedString(remoteId));
+          throw new IOException(LocalizedStrings.TCPConduit_PEER_HAS_DISAPPEARED_FROM_VIEW.toLocalizedString(memberAddress));
         } // left the view
 
         if (membershipManager.shutdownInProgress()) { // shutdown in progress
@@ -1054,12 +1030,12 @@ public class TCPConduit implements Runnable {
         if (memberInTrouble == null) {
           logger.warn(LocalizedMessage.create(
           LocalizedStrings.TCPConduit_ERROR_SENDING_MESSAGE_TO_0_WILL_REATTEMPT_1,
-          new Object[] {m, problem}));
-          memberInTrouble = m;
+          new Object[] {memberAddress, problem}));
+          memberInTrouble = memberAddress;
         }
         else {
           if (logger.isDebugEnabled()) {
-            logger.debug("Error sending message to {}", m, problem);
+            logger.debug("Error sending message to {}", memberAddress, problem);
           }
         }
 
@@ -1073,7 +1049,7 @@ public class TCPConduit implements Runnable {
             throw (IOException)problem;
           }
           else {
-            IOException ioe = new IOException( LocalizedStrings.TCPConduit_PROBLEM_CONNECTING_TO_0.toLocalizedString(remoteId));
+            IOException ioe = new IOException( LocalizedStrings.TCPConduit_PROBLEM_CONNECTING_TO_0.toLocalizedString(memberAddress));
             ioe.initCause(problem);
             throw ioe;
           }
@@ -1089,8 +1065,8 @@ public class TCPConduit implements Runnable {
             LocalizedStrings.TCPConduit_SUCCESSFULLY_RECONNECTED_TO_MEMBER_0,
             memberInTrouble));
         if (logger.isTraceEnabled()) {
-          logger.trace("new connection is {} remoteId={} memberAddress={}", conn, remoteId, memberAddress);
-      }
+          logger.trace("new connection is {} memberAddress={}", conn, memberAddress);
+        }
       }
       return conn;
       }
@@ -1102,180 +1078,6 @@ public class TCPConduit implements Runnable {
     } // for(;;)
   }
 
-//   /**
-//    * Send a message.
-//    * @return the connection used to send the message
-//    * @throws IOException if peer departed view or shutdown in progress
-//    */
-//   private Connection send(Stub remoteId, ByteBuffer bb, boolean preserveOrder, DistributionMessage msg)
-//     throws java.io.IOException
-//   {
-//     if (stopped) {
-//       throw new ConduitStoppedException("The conduit is stopped");
-//     }
-
-//     if (!QUIET) {
-//       LogWriterI18n l = getLogger();
-//       if (l.finerEnabled()) {
-//         l.finer(id.toString() + " sending " + bb
-//                 + " to " + remoteId);
-//       }
-//     }
-
-//     Connection conn = null;
-//     InternalDistributedMember memberInTrouble = null;
-//     for (;;) {
-//       // If this is the second time through this loop, we had
-//       // problems.  Tear down the connection so that it gets
-//       // rebuilt.
-//       if (conn != null) { // not first time in loop
-//         // Consult with the membership manager; if member has gone away,
-//         // there will not be an entry for this stub.
-//         InternalDistributedMember m = membershipManager.getMemberForStub(remoteId);
-//         if (m == null) {
-//           // OK, the member left.  Just register an error.
-//           throw new IOException("TCP/IP connection lost and member no longer in view");
-//         }
-//         // bug35953: Member is still in view; we MUST NOT give up!
-        
-//         // Pause just a tiny bit...
-//         try {
-//           Thread.sleep(5000);
-//         }
-//         catch (InterruptedException e) {
-//           Thread.currentThread().interrupt();
-//           if (membershipManager.shutdownInProgress()) { // shutdown in progress
-//             // Bracket our original warning
-//             if (memberInTrouble != null) {
-//               logger.info("Ending retry attempt because shutdown has started.");
-//             }
-//             throw new IOException("Abandoned because shutdown is in progress");
-//           } // shutdown in progress
-          
-//           // Strange random interrupt intercepted?
-//           logger.warning("Thread has been interrupted but no shutdown in progress", e);
-//           throw new DistributedSystemDisconnectedException(e);
-//         }
-        
-//         // Print a warning (once)
-//         if (memberInTrouble == null) {
-//           memberInTrouble = m;
-//           getLogger().warning("Attempting TCP/IP reconnect to " + memberInTrouble); 
-//         }
-//         else {
-//           getLogger().fine("Attempting TCP/IP reconnect to " + memberInTrouble);
-//         }
-        
-//         // Close the connection (it will get rebuilt later).
-//         this.stats.incReconnectAttempts();
-//         try { 
-//           conn.closeForReconnect("closing before retrying"); 
-//           } 
-//         catch (CancelException ex) {
-//           // In general we ignore close problems, but if the system
-//           // is shutting down, we should just quit.
-//           throw ex;
-//         }
-//         catch (Exception ex) {
-//           }
-//       } // not first time in loop
-      
-//       // Do the send
-//       Exception problem = null;
-//       try {
-//         // Get (or regenerate) the connection
-//         // bug36202: this could generate a ConnectionException, so it
-//         // must be caught and retried
-//         conn = getConTable().get(remoteId, preserveOrder);
-//         //      getLogger().info ("connections returned " + conn);
-//         if (conn == null) {
-//           // conduit may be closed - otherwise an ioexception would be thrown
-//           throw new IOException("Unable to reconnect to server; possible shutdown: " 
-//               + remoteId);
-//         }
-
-//         conn.sendPreserialized(bb, msg);
-//       }
-//       catch (ConnectionException e) {
-//         // Race condition between acquiring the connection and attempting
-//         // to use it: another thread closed it.
-//         problem = e;
-//       }
-//       catch (IOException e) {
-//         problem = e;
-//       }
-
-//       if (problem != null) {
-//         // Some problems are not recoverable; check an error out early.
-//         InternalDistributedMember m = membershipManager.getMemberForStub(remoteId);
-//         if (m == null) { // left the view
-//           // Bracket our original warning
-//           if (memberInTrouble != null) {
-//             logger.info("Ending retry attempt because " + memberInTrouble 
-//                 + " has disappeared.");
-//           }
-//           throw new IOException("Peer has disappeared from view");
-//         } // left the view
-        
-//         if (membershipManager.shutdownInProgress()) { // shutdown in progress
-//           // Bracket our original warning
-//           if (memberInTrouble != null) {
-//             logger.info("Ending retry attempt because shutdown has started.");
-//           }
-//           throw new IOException("Abandoned because shutdown is in progress");
-//         } // shutdown in progress
-        
-//         if (endpointRemoved(remoteId)) { // endpoint removed
-//           // TODO what does this mean?
-//           // Bracket our original warning
-//           if (memberInTrouble != null) {
-//             logger.info("Ending retry attempt because " + memberInTrouble 
-//                 + " has lost its endpoint.");
-//           }
-//           throw new IOException("Endpoint was removed");
-//         } // endpoint removed
-        
-//         // Log the warning.  We wait until now, because we want
-//         // to have m defined for a nice message...
-//         if (memberInTrouble == null) {
-//           logger.warning(
-//               "Error sending message to " + m + " (will reattempt): " 
-//                   + problem.toString(), 
-//               logger.finerEnabled() ? problem : null);
-//           memberInTrouble = m;
-//         }
-//         else {
-//           logger.fine("Error sending message to " + m, problem);
-//         }
-        
-//         // Retry the operation (indefinitely)
-//         continue;
-//       } // problem != null
-//       // Success!
-      
-//       // Make sure our logging is bracketed if there was a problem
-//       if (memberInTrouble != null) {
-//         logger.info("Successfully reestablished connection to server " 
-//             + memberInTrouble);
-//       }
-//       return conn;
-//     } // while retry
-//   }
-
-//   /**
-//    * Sends an already serialized message in a byte buffer
-//    * to the given endpoint. Waits for the send to complete
-//    * before returning.
-//    * @return the connection used to send the message
-//    */
-//   public Connection sendSync(Stub remoteId, ByteBuffer bb, int processorType, DistributionMessage msg)
-//     throws java.io.IOException
-//   {
-//     return send(remoteId, bb,
-//                 processorType == DistributionManager.SERIAL_EXECUTOR,
-//                 msg);
-//   }
-
   @Override
   public String toString() {
     return "" + id;
@@ -1301,22 +1103,22 @@ public class TCPConduit implements Runnable {
     return directChannel.getDM();
   }
   /**
-   * Closes any connections used to communicate with the given stub
+   * Closes any connections used to communicate with the given member
    */
-  public void removeEndpoint(Stub stub, String reason) {
-    removeEndpoint(stub, reason, true);
+  public void removeEndpoint(DistributedMember mbr, String reason) {
+    removeEndpoint(mbr, reason, true);
   }
   
-  public void removeEndpoint(Stub stub, String reason, boolean notifyDisconnect) {
+  public void removeEndpoint(DistributedMember mbr, String reason, boolean notifyDisconnect) {
     ConnectionTable ct = this.conTable;
     if (ct == null) {
       return;
     }
-    ct.removeEndpoint(stub, reason, notifyDisconnect);
+    ct.removeEndpoint(mbr, reason, notifyDisconnect);
   }
   
   /** check to see if there are still any receiver threads for the given end-point */
-  public boolean hasReceiversFor(Stub endPoint) {
+  public boolean hasReceiversFor(DistributedMember endPoint) {
     ConnectionTable ct = this.conTable;
     return (ct != null) && ct.hasReceiversFor(endPoint);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
index 1f411bb..773ef38 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
@@ -38,7 +38,6 @@ 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.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
@@ -46,13 +45,10 @@ import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.distributed.internal.membership.gms.MembershipManagerHelper;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
-import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.tcp.Stub;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
-import dunit.SerializableCallable;
 import dunit.SerializableRunnable;
 import dunit.VM;
 
@@ -192,7 +188,7 @@ public class DistributionManagerDUnitTest extends DistributedTestCase {
       sys.getLogWriter().info("<ExpectedException action=add>attempt to add old member</ExpectedException>");
       sys.getLogWriter().info("<ExpectedException action=add>Removing shunned GemFire node</ExpectedException>");
       try {
-        boolean accepted = mgr.addSurpriseMember(mbr, new Stub());
+        boolean accepted = mgr.addSurpriseMember(mbr);
         Assert.assertTrue("member with old ID was not rejected (bug #44566)", !accepted);
       } finally {
         sys.getLogWriter().info("<ExpectedException action=remove>attempt to add old member</ExpectedException>");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
index 44e1b46..ddbda0b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
@@ -16,8 +16,18 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.mgr;
 
-import static org.mockito.Mockito.*;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -36,7 +46,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
-import com.gemstone.gemfire.distributed.internal.AdminMessageType;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -44,7 +53,6 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.HighPriorityAckedMessage;
-import com.gemstone.gemfire.distributed.internal.HighPriorityDistributionMessage;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.MembershipListener;
 import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
@@ -60,19 +68,11 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.SuspectMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Authenticator;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.HealthMonitor;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.JoinLeave;
-import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Messenger;
-import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
-import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager.StartupEvent;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.admin.remote.AdminRequest;
-import com.gemstone.gemfire.internal.admin.remote.AdminResponse;
 import com.gemstone.gemfire.internal.admin.remote.AlertListenerMessage;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.internal.tcp.ConnectExceptions;
-import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -295,13 +295,6 @@ public class GMSMembershipManagerJUnitTest {
     suspectMember = mockMembers[1];
     manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember, "testing"));
     verify(listener).memberSuspect(suspectMember, mockMembers[0], "testing");
-    
-    InternalDistributedMember mbr = manager.getMemberForStub(new Stub(myMemberId.getInetAddress(), 2033, 20), false);
-    assertTrue(mbr == null);
-    myMemberId.setDirectChannelPort(2033);
-    mbr = manager.getMemberForStub(new Stub(myMemberId.getInetAddress(), 2033, 20), false);
-    assertTrue(mbr != null);
-    assertEquals(mbr, myMemberId);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
index 78c462f..ffd5092 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
@@ -22,6 +22,7 @@ import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.verify;
 
 import java.io.InputStream;
+import java.net.InetSocketAddress;
 import java.net.Socket;
 
 import org.junit.Test;
@@ -61,7 +62,7 @@ public class ConnectionJUnitTest {
     when(stopper.cancelInProgress()).thenReturn(null);
     when(conduit.getCancelCriterion()).thenReturn(stopper);
 
-    when(conduit.getId()).thenReturn(new Stub(SocketCreator.getLocalHost(), 10337, 1));
+    when(conduit.getId()).thenReturn(new InetSocketAddress(SocketCreator.getLocalHost(), 10337));
     
     // NIO can't be mocked because SocketChannel has a final method that
     // is used by Connection - configureBlocking


[50/50] [abbrv] incubator-geode git commit: GEODE-14: Fixed missing ChildFirstURLClassLoader

Posted by je...@apache.org.
GEODE-14: Fixed missing ChildFirstURLClassLoader


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

Branch: refs/heads/feature/GEODE-14
Commit: 71e1b1fff5a5ed196315919c6a64262652b96fab
Parents: cf95428
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Dec 21 10:03:28 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 21 10:03:28 2015 -0800

----------------------------------------------------------------------
 .../session/filter/ChildFirstClassLoader.java   | 77 --------------------
 .../modules/junit/ChildFirstClassLoader.java    | 76 +++++++++++++++++++
 .../modules/junit/PerTestClassLoaderRunner.java |  2 +-
 3 files changed, 77 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/71e1b1ff/modules/gemfire-modules-session-external/src/test/java/com/gemstone/gemfire/modules/session/filter/ChildFirstClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/gemfire-modules-session-external/src/test/java/com/gemstone/gemfire/modules/session/filter/ChildFirstClassLoader.java b/modules/gemfire-modules-session-external/src/test/java/com/gemstone/gemfire/modules/session/filter/ChildFirstClassLoader.java
deleted file mode 100644
index a8d9a7d..0000000
--- a/modules/gemfire-modules-session-external/src/test/java/com/gemstone/gemfire/modules/session/filter/ChildFirstClassLoader.java
+++ /dev/null
@@ -1,77 +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.modules.session.filter;
-
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLClassLoader;
-
-public class ChildFirstClassLoader extends URLClassLoader {
-
-  public ChildFirstClassLoader() {
-    super(new URL[]{});
-  }
-
-  public ChildFirstClassLoader(URL[] urls) {
-    super(urls);
-  }
-
-  public ChildFirstClassLoader(URL[] urls, ClassLoader parent) {
-    super(urls, parent);
-  }
-
-  @Override
-  public void addURL(URL url) {
-    super.addURL(url);
-  }
-
-  @Override
-  public Class loadClass(String name) throws ClassNotFoundException {
-    return loadClass(name, false);
-  }
-
-  /**
-   * We override the parent-first behavior established by
-   * java.lang.Classloader.
-   */
-  @Override
-  protected Class loadClass(String name, boolean resolve)
-      throws ClassNotFoundException {
-    Class c = null;
-
-    if (name.startsWith("com.gemstone")) {
-      // First, check if the class has already been loaded
-      c = findLoadedClass(name);
-
-      // if not loaded, search the local (child) resources
-      if (c == null) {
-        try {
-          c = findClass(name);
-        } catch (ClassNotFoundException cnfe) {
-          // ignore
-        }
-      }
-    }
-
-    // if we could not find it, delegate to parent
-    // Note that we don't attempt to catch any ClassNotFoundException
-    if (c == null) {
-      if (getParent() != null) {
-        c = getParent().loadClass(name);
-      } else {
-        c = getSystemClassLoader().loadClass(name);
-      }
-    }
-
-    if (resolve) {
-      resolveClass(c);
-    }
-
-    return c;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/71e1b1ff/modules/gemfire-modules-session-integration-test/src/test/java/com/gemstone/gemfire/modules/junit/ChildFirstClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/gemfire-modules-session-integration-test/src/test/java/com/gemstone/gemfire/modules/junit/ChildFirstClassLoader.java b/modules/gemfire-modules-session-integration-test/src/test/java/com/gemstone/gemfire/modules/junit/ChildFirstClassLoader.java
new file mode 100644
index 0000000..b566d05
--- /dev/null
+++ b/modules/gemfire-modules-session-integration-test/src/test/java/com/gemstone/gemfire/modules/junit/ChildFirstClassLoader.java
@@ -0,0 +1,76 @@
+/*=========================================================================
+ * 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.modules.junit;
+
+import java.net.URL;
+import java.net.URLClassLoader;
+
+public class ChildFirstClassLoader extends URLClassLoader {
+
+  public ChildFirstClassLoader() {
+    super(new URL[]{});
+  }
+
+  public ChildFirstClassLoader(URL[] urls) {
+    super(urls);
+  }
+
+  public ChildFirstClassLoader(URL[] urls, ClassLoader parent) {
+    super(urls, parent);
+  }
+
+  @Override
+  public void addURL(URL url) {
+    super.addURL(url);
+  }
+
+  @Override
+  public Class loadClass(String name) throws ClassNotFoundException {
+    return loadClass(name, false);
+  }
+
+  /**
+   * We override the parent-first behavior established by
+   * java.lang.Classloader.
+   */
+  @Override
+  protected Class loadClass(String name, boolean resolve)
+      throws ClassNotFoundException {
+    Class c = null;
+
+    if (name.startsWith("com.gemstone")) {
+      // First, check if the class has already been loaded
+      c = findLoadedClass(name);
+
+      // if not loaded, search the local (child) resources
+      if (c == null) {
+        try {
+          c = findClass(name);
+        } catch (ClassNotFoundException cnfe) {
+          // ignore
+        }
+      }
+    }
+
+    // if we could not find it, delegate to parent
+    // Note that we don't attempt to catch any ClassNotFoundException
+    if (c == null) {
+      if (getParent() != null) {
+        c = getParent().loadClass(name);
+      } else {
+        c = getSystemClassLoader().loadClass(name);
+      }
+    }
+
+    if (resolve) {
+      resolveClass(c);
+    }
+
+    return c;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/71e1b1ff/modules/gemfire-modules-session-integration-test/src/test/java/com/gemstone/gemfire/modules/junit/PerTestClassLoaderRunner.java
----------------------------------------------------------------------
diff --git a/modules/gemfire-modules-session-integration-test/src/test/java/com/gemstone/gemfire/modules/junit/PerTestClassLoaderRunner.java b/modules/gemfire-modules-session-integration-test/src/test/java/com/gemstone/gemfire/modules/junit/PerTestClassLoaderRunner.java
index 254ff3d..fce6471 100644
--- a/modules/gemfire-modules-session-integration-test/src/test/java/com/gemstone/gemfire/modules/junit/PerTestClassLoaderRunner.java
+++ b/modules/gemfire-modules-session-integration-test/src/test/java/com/gemstone/gemfire/modules/junit/PerTestClassLoaderRunner.java
@@ -95,7 +95,7 @@ public class PerTestClassLoaderRunner extends NamedRunner {
       }
     }
 
-    ClassLoader classLoader = new ChildFirstURLClassLoader(
+    ClassLoader classLoader = new ChildFirstClassLoader(
         urls.toArray(new URL[]{}),
         Thread.currentThread().getContextClassLoader()
     );


[11/50] [abbrv] incubator-geode git commit: GEODE-654: Changes LIFO queue to remove touched entry

Posted by je...@apache.org.
GEODE-654: Changes LIFO queue to remove touched entry

Removes any entry that has been touched while processing
the LIFO queue. Prevents issue when eviction gets stuck
during startup and GII and an entry can not be removed.


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

Branch: refs/heads/feature/GEODE-14
Commit: f7670e1688a95a026a0376724bfe2277cb16eb94
Parents: fe29594
Author: Vince Ford <vf...@apache.org>
Authored: Fri Dec 11 11:30:35 2015 -0800
Committer: Vince Ford <vf...@apache.org>
Committed: Fri Dec 11 16:30:36 2015 -0800

----------------------------------------------------------------------
 .../internal/cache/lru/NewLIFOClockHand.java    |  78 ++++++++----
 .../internal/cache/lru/NewLRUClockHand.java     |  14 ++-
 ...victionAlgoMemoryEnabledRegionJUnitTest.java | 119 +++++++++++++++++++
 3 files changed, 185 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7670e16/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLIFOClockHand.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLIFOClockHand.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLIFOClockHand.java
index eac00c4..2ebc7a0 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLIFOClockHand.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLIFOClockHand.java
@@ -18,6 +18,7 @@ package com.gemstone.gemfire.internal.cache.lru;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
+import com.gemstone.gemfire.internal.cache.AbstractRegionEntry;
 
 /**
  * NewLIFOClockHand holds the behavior for LIFO logic , Overwriting
@@ -39,30 +40,59 @@ public class NewLIFOClockHand extends NewLRUClockHand {
       ,NewLRUClockHand oldList){
     super(region,ccHelper,oldList);
   }
+  
+    /* Fetch the tail member which should be the last added value and remove it
+     * from the list
+     */
+    protected LRUClockNode getTailEntry() {
+        LRUClockNode aNode = null;
+        synchronized (this.lock) {
+            aNode = this.tail.prevLRUNode();
+            if (aNode == this.head) {
+                return null; //end of lru list
+            }
+            //remove entry from list
+            LRUClockNode prev = aNode.prevLRUNode();
+            prev.setNextLRUNode(this.tail);
+            this.tail.setPrevLRUNode(prev);
+            aNode.setNextLRUNode(null);
+            aNode.setPrevLRUNode(null);
+            super.size--;
+        }
+        return aNode;
+    }
 
-  /*
-   *  return the Entry that is considered most recently used
-   */
-  @Override
-   public LRUClockNode getLRUEntry() { // new getLIFOEntry
-    LRUClockNode aNode = null;
-    synchronized (this.lock) {
-      aNode = this.tail.prevLRUNode();
-      if(aNode == this.head) {
-        return null;
-      }
-      //TODO - Dan 9/23/09 We should probably
-      //do something like this to change the tail pointer.
-      //But this code wasn't changing the tail before
-      //I made this a doubly linked list, and I don't
-      //want to change it on this branch.
-//      LRUClockNode prev = aNode.prevLRUNode();
-//      prev.setNextLRUNode(this.tail);
-//      aNode.setNextLRUNode(null);
-//      aNode.setPrevLRUNode(null);
+    /*
+     *  return the Entry that is considered most recently used and available 
+     * to be evicted to overflow
+     */
+    @Override
+    public LRUClockNode getLRUEntry() {
+        long numEvals = 0;
+        LRUClockNode aNode = null;
+        //search for entry to return from list
+        for (;;) {
+            aNode = getTailEntry();
+            //end of Lifo list stop searching
+            if (aNode == null) {
+                break;
+            }
+            numEvals++;
+            synchronized (aNode) {
+                //look for another entry if in transaction
+		boolean inUseByTransaction = false;
+                if (aNode instanceof AbstractRegionEntry) {
+                    if (((AbstractRegionEntry) aNode).isInUseByTransaction()) {
+                       inUseByTransaction=true; 
+                    }
+                }
+                //if entry NOT used by transaction and NOT evicted return entry
+                if (!inUseByTransaction && !aNode.testEvicted()) {
+                    break;
+                }
+            }
+        }
+        this.stats().incEvaluations(numEvals);
+        return aNode;
     }
-    /* no need to update stats here as when this function finished executing 
-       next few calls update stats */
-    return aNode.testEvicted()? null:aNode;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7670e16/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java
index b16cd91..59245f8 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/lru/NewLRUClockHand.java
@@ -55,7 +55,7 @@ public class NewLRUClockHand  {
   /**  Description of the Field */
   final private LRUStatistics stats;
   /** Counter for the size of the LRU list */
-  private int size = 0;
+  protected int size = 0;
   
 public static final boolean debug = Boolean.getBoolean("gemfire.verbose-lru-clock");
 
@@ -185,7 +185,7 @@ public NewLRUClockHand(Object region, EnableLRU ccHelper,
       
       aNode.setNextLRUNode(null);
       aNode.setPrevLRUNode(null);
-      this.size++;
+      this.size--;
       return aNode;
     }
   }
@@ -334,6 +334,7 @@ public NewLRUClockHand(Object region, EnableLRU ccHelper,
       prev.setNextLRUNode(next);
       entry.setNextLRUNode(null);
       entry.setPrevLRUNode(null);
+      this.size--;
     }
     return true;
   }
@@ -386,6 +387,15 @@ public NewLRUClockHand(Object region, EnableLRU ccHelper,
     this.size = 0;
   }
   
+    /**
+     * Get size of LRU queue
+     *
+     * @return size
+     */
+    public int size() {
+        return size;
+    }
+
   /** perform work of clear(), after subclass has properly synchronized */
 //  private void internalClear() {
 //    stats().resetCounter();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f7670e16/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
index e33c6ac..c8ffd7c 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/LIFOEvictionAlgoMemoryEnabledRegionJUnitTest.java
@@ -46,6 +46,8 @@ import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.internal.cache.lru.LRUStatistics;
 import com.gemstone.gemfire.internal.cache.lru.NewLRUClockHand;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.internal.cache.lru.LRUClockNode;
+import com.gemstone.gemfire.internal.cache.lru.EnableLRU;
 
 /**
  * This is a test verifies region is LIFO enabled by MEMORY verifies correct
@@ -305,6 +307,53 @@ public class LIFOEvictionAlgoMemoryEnabledRegionJUnitTest {
       fail("Test failed");
     }
   }
+  
+//Basic checks to validate lifo queue implementation works as expected
+    @Test
+    public void testLIFOQueue() {
+        try {
+            assertNotNull(cache);
+            Region rgn = cache.getRegion(Region.SEPARATOR + regionName);
+            assertNotNull(rgn);
+            //insert data
+            lifoClockHand.appendEntry(new TestLRUNode(1));
+            lifoClockHand.appendEntry(new TestLRUNode(2));
+            lifoClockHand.appendEntry(new TestLRUNode(3));
+            assertTrue(lifoClockHand.size() == 3);
+            //make sure data is removed in LIFO fashion
+            TestLRUNode tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
+            assertTrue("Value = " + tailValue.getValue(), tailValue.getValue() == 3);
+            assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 2);
+            tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
+            assertTrue("Value = " + tailValue.getValue(), tailValue.getValue() == 2);
+            assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 1);
+            tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
+            assertTrue("Value = " + tailValue.getValue(), tailValue.getValue() == 1);
+            assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 0);
+            tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
+            assertTrue("No Value - null", tailValue == null);
+            assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 0);
+            //check that entries not available or already evicted are skipped and removed
+            TestLRUNode testlrunode = new TestLRUNode(1);
+            lifoClockHand.appendEntry(testlrunode);
+            testlrunode = new TestLRUNode(2);
+            testlrunode.setEvicted();
+            lifoClockHand.appendEntry(testlrunode);
+            testlrunode = new TestLRUNode(3);
+            testlrunode.setEvicted();
+            lifoClockHand.appendEntry(testlrunode);
+            tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
+            assertTrue("Value = " + tailValue.getValue(), tailValue.getValue() == 1);
+            assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 0);
+            tailValue = (TestLRUNode) lifoClockHand.getLRUEntry();
+            assertTrue("No Value - null", tailValue == null);
+            assertTrue("LIFO Queue Size = " + lifoClockHand.size(), lifoClockHand.size() == 0);
+            //TODO : need tests for data still part of transaction 
+        } catch (Exception ex) {
+            ex.printStackTrace();
+            fail(ex.getMessage());
+        }
+    }
 
   
   // purpose to create object ,size of byteArraySize 
@@ -315,3 +364,73 @@ public class LIFOEvictionAlgoMemoryEnabledRegionJUnitTest {
   }
 }
 
+//test class for validating LIFO queue
+class TestLRUNode implements LRUClockNode{
+  
+    LRUClockNode next=null;
+    LRUClockNode prev=null;
+    boolean evicted=false;
+    boolean recentlyUsed=false;
+    int value=0;
+    
+  public TestLRUNode(int value){
+      this.value=value;
+  }
+  
+  public int getValue(){ 
+      return value;
+  }
+  
+  public void setNextLRUNode( LRUClockNode next ){
+      this.next=next;
+  }
+  
+  public void setPrevLRUNode( LRUClockNode prev ){
+      this.prev=prev;
+  }
+  
+  public LRUClockNode nextLRUNode(){
+      return next;
+  }
+  
+  public LRUClockNode prevLRUNode(){
+      return prev;
+  }
+
+  public int updateEntrySize(EnableLRU ccHelper){
+      return 0;
+  }
+ 
+  public int updateEntrySize(EnableLRU ccHelper, Object value){
+      return 0;
+  }
+  
+  public int getEntrySize(){
+      return 0;
+  }
+  
+  public boolean testRecentlyUsed(){
+      return recentlyUsed;
+  }
+  
+  public void setRecentlyUsed(){
+      recentlyUsed=true;
+  }
+  
+  public void unsetRecentlyUsed(){
+      recentlyUsed=false;
+  }
+  
+  public void setEvicted(){
+      evicted=true;
+  }
+  
+  public void unsetEvicted(){
+      evicted=false;
+  }
+  
+  public boolean testEvicted(){
+      return evicted;
+  } 
+}
+


[27/50] [abbrv] incubator-geode git commit: GEODE-568: wait at most 5 seconds before checking the size of the internal region to allow eviction to complete

Posted by je...@apache.org.
GEODE-568: wait at most 5 seconds before checking the size of the internal region to allow eviction to complete


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

Branch: refs/heads/feature/GEODE-14
Commit: d58588a44a536c12adf0c5dcb64beda06b18e1f3
Parents: ec9d16a
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Dec 14 15:14:15 2015 -0800
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Dec 14 15:14:15 2015 -0800

----------------------------------------------------------------------
 .../gemfire/management/CacheManagementDUnitTest.java     | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d58588a4/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java
index 19fa5d6..b9762a4 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/CacheManagementDUnitTest.java
@@ -23,6 +23,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
 
 import javax.management.InstanceNotFoundException;
 import javax.management.JMException;
@@ -44,6 +46,8 @@ import com.gemstone.gemfire.management.internal.ManagementConstants;
 import com.gemstone.gemfire.management.internal.SystemManagementService;
 import com.gemstone.gemfire.management.internal.NotificationHub.NotificationHubListener;
 
+import static com.jayway.awaitility.Awaitility.*;
+import static org.hamcrest.Matchers.*;
 import dunit.AsyncInvocation;
 import dunit.DistributedTestCase;
 import dunit.SerializableRunnable;
@@ -938,10 +942,9 @@ public class CacheManagementDUnitTest extends ManagementTestBase {
         // Even though we got 15 notification only 10 should be there due to
         // eviction attributes set in notification region
 
-        assertEquals(10, member1NotifRegion.size());
-        assertEquals(10, member2NotifRegion.size());
-        assertEquals(10, member3NotifRegion.size());
-
+        waitAtMost(5, TimeUnit.SECONDS).untilCall(to(member1NotifRegion).size(), equalTo(10));
+        waitAtMost(5, TimeUnit.SECONDS).untilCall(to(member2NotifRegion).size(), equalTo(10));
+        waitAtMost(5, TimeUnit.SECONDS).untilCall(to(member3NotifRegion).size(), equalTo(10));
       }
     });
 


[19/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/font/fontawesome-webfont-woff.woff
----------------------------------------------------------------------
diff --git a/gemfire-site/content/font/fontawesome-webfont-woff.woff b/gemfire-site/content/font/fontawesome-webfont-woff.woff
deleted file mode 100755
index b9bd17e..0000000
Binary files a/gemfire-site/content/font/fontawesome-webfont-woff.woff and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/img/apache_geode_logo.png
----------------------------------------------------------------------
diff --git a/gemfire-site/content/img/apache_geode_logo.png b/gemfire-site/content/img/apache_geode_logo.png
deleted file mode 100644
index 14b6ac0..0000000
Binary files a/gemfire-site/content/img/apache_geode_logo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/img/apache_geode_logo_white.png
----------------------------------------------------------------------
diff --git a/gemfire-site/content/img/apache_geode_logo_white.png b/gemfire-site/content/img/apache_geode_logo_white.png
deleted file mode 100644
index 2a0cda8..0000000
Binary files a/gemfire-site/content/img/apache_geode_logo_white.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/img/apache_geode_logo_white_small.png
----------------------------------------------------------------------
diff --git a/gemfire-site/content/img/apache_geode_logo_white_small.png b/gemfire-site/content/img/apache_geode_logo_white_small.png
deleted file mode 100644
index bf8aaa0..0000000
Binary files a/gemfire-site/content/img/apache_geode_logo_white_small.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/img/check_flat/default.png
----------------------------------------------------------------------
diff --git a/gemfire-site/content/img/check_flat/default.png b/gemfire-site/content/img/check_flat/default.png
deleted file mode 100755
index 5a89765..0000000
Binary files a/gemfire-site/content/img/check_flat/default.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/img/egg-logo.png
----------------------------------------------------------------------
diff --git a/gemfire-site/content/img/egg-logo.png b/gemfire-site/content/img/egg-logo.png
deleted file mode 100644
index 2169a2a..0000000
Binary files a/gemfire-site/content/img/egg-logo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/img/github.png
----------------------------------------------------------------------
diff --git a/gemfire-site/content/img/github.png b/gemfire-site/content/img/github.png
deleted file mode 100644
index f19ee0d..0000000
Binary files a/gemfire-site/content/img/github.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/index.html b/gemfire-site/content/index.html
deleted file mode 100644
index c0d92a1..0000000
--- a/gemfire-site/content/index.html
+++ /dev/null
@@ -1,295 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-
-<head>
-    <meta charset="utf-8">
-    <title>Apache Geode (incubating) — Performance is key. Consistency is a must.</title>
-    <meta http-equiv="x-ua-compatible" content="ie=edge" />
-    <meta name="viewport" content="width=device-width, initial-scale=1, minimum-scale=1, maximum-scale=1" />
-    <meta name="keywords" content="Apache Geode, Geode, GemFire, In-memory, IMDB, IMDG, cache">
-    <meta name="description" content="Apache Geode is a distributed, in-memory database with strong data consistency, built to support transactional applications with low latency and high concurrency needs." />
-    <meta property="og:title" content="Apache Geode (incubating)" />
-    <meta property="og:description" content="Apache Geode is a distributed, in-memory database with strong data consistency, built to support transactional applications with low latency and high concurrency needs." />
-    <!-- Loading Typekit -->
-    <script type="text/javascript" src="//use.typekit.net/ddl7izx.js"></script>
-    <script type="text/javascript">try{Typekit.load();}catch(e){}</script>
-    <!-- Place this tag right after the last button or just before your close body tag. -->
-    <script async defer id="github-bjs" src="https://buttons.github.io/buttons.js"></script>
-        <!-- Loading Bootstrap -->
-    <link href="/bootstrap/bootstrap.min.css" rel="stylesheet" type='text/css'>
-    <link href="/css/bootflat.css" rel="stylesheet" type='text/css'>
-    <link href="/css/geode-site.css" rel="stylesheet" type='text/css'>
-    <link href="https://fonts.googleapis.com/css?family=Open+Sans:200,400,500,300,600,800,700,400italic,600italic,700italic,800italic,300italic" rel="stylesheet" type="text/css">
-    <link href="/css/font-awesome.min.css" rel="stylesheet" type='text/css'>
-
-    
-
-    <!-- HTML5 shim, for IE6-8 support of HTML5 elements. All other JS at the end of file. -->
-    <!--[if lt IE 9]>
-      <script src="js/html5shiv.js"></script>
-      <script src="js/respond.min.js"></script>
-    <![endif]-->
-</head>
-<body>
-
-    <header class="navbar navbar-inverse navbar-fixed-top bf-docs-nav " role="banner">
-    <div class="container">
-        <div class="navbar-header">
-            <button class="navbar-toggle" type="button" data-toggle="collapse" data-target=".bf-navbar-collapse">
-                <span class="sr-only">Toggle navigation</span>
-                <span class="icon-bar"></span>
-                <span class="icon-bar"></span>
-                <span class="icon-bar"></span>
-            </button>
-        </div>
-        <a href="/" class="navbar-brand">
-                <img id="home-logo" src="/img/apache_geode_logo_white_small.png" />
-            </a>
-        <nav class="collapse navbar-collapse bf-navbar-collapse" role="navigation">
-            <ul class="nav navbar-nav navbar-right">
-                <li class=""><a href="/community/"><span class="icns icon-group"></span></a></li>
-                <li><a href="http://geode.docs.pivotal.io" target="_blank"><span class="icns icon-book"></span></a></li>
-                <li><a href="http://github.com/apache/incubator-geode" target="_blank"><span class="icns icon-github-sign"></span></a></li>
-                <!--<li><a href="https://trello.com/b/exQmJIOn/usergrid" target="_blank"><span class="icns icon-trello"></span></a></li>-->
-                <li><a href="https://issues.apache.org/jira/browse/GEODE/"
-                       target="_blank"><span class="icns icon-bug"></span></a></li>
-                <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank"><span class="icns icon-stackexchange"></span></a></li>
-                <li><a href="/community/#mailing-lists"><span class="icns icon-envelope"></span></a></li>
-                <!-- <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li> -->
-                <li><a href="https://twitter.com/apachegeode" target="_blank"><span class="icns icon-twitter"></span></a></li>
-                <li><a href="https://cwiki.apache.org/confluence/display/geode/" target="_blank"><span class="icns icon-edit"></span></a></li>
-                <li><a href="/releases/"><span class="icns icon-releases"></span></a></li>
-            </ul>
-        </nav>
-    </div>
-    </header>
-
-
-
-
-
-<section class="bf-masthead" id="content" role="main">
-    <div class="bf-masthead-bg">
-        <div class="container">
-            <img class="logo-title img-responsive hidden-xs" src="img/apache_geode_logo.png" />
-            <div class="text-container">
-                <h2 class="tagline"><em>Performance</em> is key. <em>Consistency</em> is a must.</h2>
-                <p class="description">Solving low latency data management problems at very high concurrency since 2002.<br/>
-                  <br/>Build high-speed, data-intensive applications that elastically meet performance requirements at any scale.<br/>
-                  Take advantage of Apache Geode's unique technology that blends advanced techniques for data replication, partitioning and distributed processing.
-
-                  <br/><br/>
-                  Apache Geode (incubating) provides a database-like consistency model, reliable transaction processing and a shared-nothing architecture to maintain very low latency performance with high concurrency processing.<br/></p>
-            </div>
-
-            <div class="btn-wrapper">
-                <!-- Place this tag where you want the button to render. -->
-              <a class="github-button" href="https://github.com/apache/incubator-geode" data-icon="octicon-star" data-style="mega" data-count-href="/apache/incubator-geode/stargazers" data-count-api="/repos/apache/incubator-geode#stargazers_count" data-count-aria-label="# stargazers on GitHub" aria-label="Star apache/incubator-geode on GitHub">Star</a>
-              <a class="github-button" href="https://github.com/apache/incubator-geode/fork" data-icon="octicon-repo-forked" data-style="mega" data-count-href="/apache/incubator-geode/network" data-count-api="/repos/apache/incubator-geode#forks_count" data-count-aria-label="# forks on GitHub" aria-label="Fork apache/incubator-geode on GitHub">Fork</a>
-              <a class="github-button" href="https://github.com/apache/incubator-geode" data-icon="octicon-eye" data-style="mega" data-count-href="/apache/incubator-geode/watchers" data-count-api="/repos/apache/incubator-geode#subscribers_count" data-count-aria-label="# watchers on GitHub" aria-label="Watch apache/incubator-geode on GitHub">Watch</a>
-              <!--<p><span class="or">or</span> <a href="#">Download Geode</a></p>-->
-            </div>
-        </div>
-    </div>
-</section>
-
-<section class="bf-features">
-    <div class="container">
-        <div class="row">
-            <div class="col-md-4">
-                <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-sitemap"></i></div>
-                <h3>Replication and Partitioning</h3>
-                <p>Data can easily be partitioned (sharded) or replicated between nodes allowing performance to scale as needed. Durability is ensured through redundant in-memory copies and disk-based persistence.</p>
-            </div>
-            <div class="col-md-4">
-                <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-hdd"></i></div>
-                <h3>Persistence</h3>
-                <p>Super fast write-ahead-logging (WAL) persistence with a shared-nothing architecture that is optimized for fast parallel recovery of nodes or an entire cluster.</p>
-            </div>
-            <div class="col-md-4">
-                <div class="bf-icon-wrap"><i aria-hidden="true" class="icon-rocket"></i></div>
-                <h3>Performance</h3>
-                <p>Linear-scaling low latency for transactions, reads, writes and query processing of indexed or unindexed data.</p>
-            </div>
-        </div>
-        <div class="row">
-            <div class="col-md-4">
-                <div class="bf-icon-wrap" style="font-size:40px; vertical-align: 15px;"><i aria-hidden="true" class="icon-fast-forward"></i><i aria-hidden="true" class="icon-dashboard"></i></div>
-                <h3>In-Memory Storage</h3>
-                <p>Blazing fast in-memory storage optimized for large heaps, with the option of using off-heap storage, compression and features such as disk-overflow, eviction and expiration of data.</p>
-            </div>
-            <div class="col-md-4">
-                <div class="bf-icon-wrap"><span style="font-size:60px" aria-hidden="true" class="icon-cogs"></span></div>
-                <h3>Functions</h3>
-                <p>Distributed location-aware user functions can be deployed and executed by the same nodes storing relevant sharded data for fast parallel processing. Failed operations can be retried on replicant nodes.</p>
-            </div>
-            <div class="col-md-4">
-                <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-credit-card"></i></div>
-                <h3>Transactions</h3>
-                <p>ACID distributed transactions support efficient and safe coordinated operations on colocated data. Transactions can be initiated or suspended by either a client or a server.</p>
-            </div>
-        </div>
-        <div class="row">
-          <div class="col-md-4">
-              <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-table"></i></div>
-              <h3>OQL and Indexes</h3>
-              <p>Object Query Language allows distributed query execution on hot and cold data, with SQL-like capabilities, including joins.<br/>
-              Multiple kinds of indexes can be defined and consistently maintained across the cluster.</p>
-          </div>
-          <div class="col-md-4">
-              <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-bolt"></i></div>
-              <h3>Events</h3>
-              <p>Clients can be notified about server-side data events, and servers can react synchronously or asynchronously with guaranteed delivery of ordered events.</p>
-          </div>
-          <div class="col-md-4">
-              <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-cloud"></i></div>
-              <h3>Clustering</h3>
-              <p>Highly scalable, robust advanced clustering technology with failure detection, dynamic scaling, and network-partition detection algorithms.</p>
-          </div>
-        </div>
-    </div>
-
-    </div>
-</section>
-
-<section class="bf-questions">
-    <div class="container">
-            <div class="col-md-12 text-center cta">
-                And much more... Interested? You can check our <a href="https://cwiki.apache.org/confluence/display/GEODE/Index#Index-Geodein5minutesGeodein5minutes" target="_blank" class="btn btn-inverse btn-lg">Geode in 5 minutes tutorial</a> <span class="avoidwrap">, ask a question on the <a href="/community/" class="btn btn-inverse btn-lg">Mailing lists</a> or <a href="http://stackoverflow.com/search?q=Apache%20Geode" class="btn btn-inverse btn-lg">StackOverflow</a></span>
-            </div>
-    </div>
-</section
-
-<section class="bf-news">
-    <div class="container">
-
-        <div class="row">
-            <div class="col-md-12 text-left">
-                <h2>About the Project</h2>
-                <p>Apache Geode (incubating) is a data management platform that provides real-time, consistent access to data-intensive applications throughout widely distributed cloud architectures.</p>
-
-                <p>By pooling 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, Apache Geode is an in-memory data management system that provides reliable asynchronous event notifications and guaranteed message delivery.</p>
-
-                <p>Apache Geode is a mature, robust technology originally developed by GemStone Systems in Beaverton, Oregon.
-Commercially available as GemFire™, the technology was first widely deployed in the financial sector as the transactional, low-latency data engine used
-in Wall Street trading platforms.
-Today Apache Geode is used by over 600 enterprise customers for high-scale business applications that must meet low latency and 24x7 availability requirements.</p>
-
-                <p>This project is undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.</p>
-            </div>
-            <!--
-            <div class="col-md-4 text-left">
-                <h2>Recent Releases</h2>
-
-
-            </div>
-            -->
-        </div>
-
-    </div>
-</section>
-
-
-
-
-<footer class="bf-footer" role="contentinfo">
-    <div class="container">
-        <div class="row">
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header"><a href="/">Home</a></li>
-                    <li class="nav-header"><a href="/community/">Community</a></li>
-                        <li><a href="/community/#events">Events</a></li>
-                        <li><a href="/community/#mailing-lists">Mailing Lists</a></li>
-                        <li><a href="/community/#deployments">Deployments</a></li>
-                        <!-- <li><a href="/community/#committers">Commiters</a></li> -->
-                </ul>
-            </div>
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header"><a href="http://github.com/apache/incubator-geode" target="_blank">Code</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Project+Proposals+and+Specifications" target="_blank">Specifications</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Geode+Internal+Architecture" target="_blank">Internal Architecture</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Writing+tests" target="_blank">Writing Tests</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Criteria+for+Code+Submissions" target="_blank">Code Submissions</a></li>
-                </ul>
-            </div>
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header">Resources</li>
-                    <li><a href="http://github.com/apache/geode-incubator" target="_blank">GitHub Code</a></li>
-                    <li><a href="http://geode.docs.pivotal.io" target="_blank">Docs</a></li>
-                    <li><a href="https://issues.apache.org/jira/browse/GEODE" target="_blank">JIRA Bug Tracker</a></li>
-                    <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank">StackOverflow</a></li>
-                    <li><a href="/community/#live">Live Chat</a></li>
-                    <li><a href="https://twitter.com/apachegeode" target="_blank">Twitter</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Index#Index-Geodein5minutesGeodein5minutes" target="_blank">Geode in 5 minutes</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/How+to+Contribute" target="_blank">How to Contribute</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Application+Development" target="_blank">Application Development</a></li>
-
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Technology+FAQ" target="_blank">FAQ</a></li>
-
-                </ul>
-            </div>
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header">Apache</li>
-                    <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                    <li><a href="http://www.apache.org/foundation/sponsorship.html" target="_blank">Sponsorship</a></li>
-                    <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                    <li><a href="http://www.apache.org/security/">Security</a></li>
-                    <li><a href="http://www.apache.org/" target="_blank">Apache Foundation</a></li>
-                </ul>
-            </div>
-            <div class="col-md-4">
-              <a class="twitter-timeline" href="https://twitter.com/search?q=%23ApacheGeode%20OR%20Apache%20Geode" data-widget-id="665030887004725248">Tweets about #ApacheGeode OR Apache Geode</a>
-<script>!function(d,s,id){var js,fjs=d.getElementsByTagName(s)[0],p=/^http:/.test(d.location)?'http':'https';if(!d.getElementById(id)){js=d.createElement(s);js.id=id;js.src=p+"://platform.twitter.com/widgets.js";fjs.parentNode.insertBefore(js,fjs);}}(document,"script","twitter-wjs");</script>
-            </div>
-        <!--
-            <div class="col-md-4">
-                <div class="input-group form-search">
-                    <input type="text" class="form-control search-query">
-                    <span class="input-group-btn">
-                        <button type="submit" class="btn btn-primary" data-type="last">Search</button>
-                    </span>
-                </div>
-            </div> -->
-        </div>
-        <div class="row">
-          <center>
-            <div id="copyright">
-                <a href="http://incubator.apache.org" target="_blank"><img src="/img/egg-logo.png" /></a><br/><br/>
-                <p class="credits">
-                Apache Geode is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
-                </p>
-                <p>Copyright © 2015 The Apache Software Foundation, Licensed under the Apache License, Version 2.0.<br>
-                Apache, Apache Geode (incubating), and the Apache feather logos are trademarks of The Apache Software Foundation.</p>
-                <p>Site designed & assembled with love by <a href="https://github.com/ryuneeee">@ryuneeee</a> + <a href="https://github.com/realbeast">@realbeast</a> + <a href="https://twitter.com/timanglade">@timanglade</a> + <a href="https://twitter.com/snoopdave">@snoopdave</a> for Apache Usergrid.</p>
-                <p>Modified for Apache Geode by <a href="https://twitter.com/william_markito">@william_markito</a>.</p>
-            </div>
-          </center>
-        </div>
-    </div>
-</footer>
-
-<script type="text/javascript" src="/js/head.js"></script>
-<script type="text/javascript">
-    head.js("/js/jquery-1.10.1.min.js", "/js/bootstrap.min.js", "/js/usergrid-site.js");
-</script>
-<!-- update the code below for google analytics -->
-<!--
- <script>
-  (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
-  (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
-  m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
-  })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
-
-  ga('create', 'UA-45815079-1', 'apache.org');
-  ga('send', 'pageview');
-
-</script> -->
-
-
-
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/js/bootstrap.min.js
----------------------------------------------------------------------
diff --git a/gemfire-site/content/js/bootstrap.min.js b/gemfire-site/content/js/bootstrap.min.js
deleted file mode 100644
index 3137285..0000000
--- a/gemfire-site/content/js/bootstrap.min.js
+++ /dev/null
@@ -1,8 +0,0 @@
-/**
-* Bootstrap v3.0.0 by @fat and @mdo
-* Copyright 2013 Twitter Inc.
-* Licensed under http://www.apache.org/licenses/LICENSE-2.0.
-*
-* Designed and built with all the love in the world by @mdo and @fat.
-*/
-if(!jQuery)throw new Error("Bootstrap requires jQuery");+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]}}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one(a.support.transition.end,function(){c=!0});var e=function(){c||a(d).trigger(a.support.transition.end)};return setTimeout(e,b),this},a(function(){a.support.transition=b()})}(window.jQuery),+function(a){"use strict";var b='[data-dismiss="alert"]',c=function(c){a(c).on("click",b,this.close)};c.prototype.close=function(b){function c(){f.trigger("closed.bs.alert").remove()}var d=a(this),e=d.attr("data-target");e||(e=d.attr("href"),e=e&&e.replace(/.*(?=#[^\s]*$)/,""));var f=a(e);b&&b.preventDefault(),f.length||(f=d.hasClass("alert")?d:d.parent()),f.trigger(b=a.Event("close.bs.alert")),b.isDefau
 ltPrevented()||(f.removeClass("in"),a.support.transition&&f.hasClass("fade")?f.one(a.support.transition.end,c).emulateTransitionEnd(150):c())};var d=a.fn.alert;a.fn.alert=function(b){return this.each(function(){var d=a(this),e=d.data("bs.alert");e||d.data("bs.alert",e=new c(this)),"string"==typeof b&&e[b].call(d)})},a.fn.alert.Constructor=c,a.fn.alert.noConflict=function(){return a.fn.alert=d,this},a(document).on("click.bs.alert.data-api",b,c.prototype.close)}(window.jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d)};b.DEFAULTS={loadingText:"loading..."},b.prototype.setState=function(a){var b="disabled",c=this.$element,d=c.is("input")?"val":"html",e=c.data();a+="Text",e.resetText||c.data("resetText",c[d]()),c[d](e[a]||this.options[a]),setTimeout(function(){"loadingText"==a?c.addClass(b).attr(b,b):c.removeClass(b).removeAttr(b)},0)},b.prototype.toggle=function(){var a=this.$element.closest('[data-toggle="buttons"]');if(a.l
 ength){var b=this.$element.find("input").prop("checked",!this.$element.hasClass("active")).trigger("change");"radio"===b.prop("type")&&a.find(".active").removeClass("active")}this.$element.toggleClass("active")};var c=a.fn.button;a.fn.button=function(c){return this.each(function(){var d=a(this),e=d.data("bs.button"),f="object"==typeof c&&c;e||d.data("bs.button",e=new b(this,f)),"toggle"==c?e.toggle():c&&e.setState(c)})},a.fn.button.Constructor=b,a.fn.button.noConflict=function(){return a.fn.button=c,this},a(document).on("click.bs.button.data-api","[data-toggle^=button]",function(b){var c=a(b.target);c.hasClass("btn")||(c=c.closest(".btn")),c.button("toggle"),b.preventDefault()})}(window.jQuery),+function(a){"use strict";var b=function(b,c){this.$element=a(b),this.$indicators=this.$element.find(".carousel-indicators"),this.options=c,this.paused=this.sliding=this.interval=this.$active=this.$items=null,"hover"==this.options.pause&&this.$element.on("mouseenter",a.proxy(this.pause,this))
 .on("mouseleave",a.proxy(this.cycle,this))};b.DEFAULTS={interval:5e3,pause:"hover",wrap:!0},b.prototype.cycle=function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},b.prototype.getActiveIndex=function(){return this.$active=this.$element.find(".item.active"),this.$items=this.$active.parent().children(),this.$items.index(this.$active)},b.prototype.to=function(b){var c=this,d=this.getActiveIndex();return b>this.$items.length-1||0>b?void 0:this.sliding?this.$element.one("slid",function(){c.to(b)}):d==b?this.pause().cycle():this.slide(b>d?"next":"prev",a(this.$items[b]))},b.prototype.pause=function(b){return b||(this.paused=!0),this.$element.find(".next, .prev").length&&a.support.transition.end&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),this.interval=clearInterval(this.interval),this},b.prototype.next=function(){return th
 is.sliding?void 0:this.slide("next")},b.prototype.prev=function(){return this.sliding?void 0:this.slide("prev")},b.prototype.slide=function(b,c){var d=this.$element.find(".item.active"),e=c||d[b](),f=this.interval,g="next"==b?"left":"right",h="next"==b?"first":"last",i=this;if(!e.length){if(!this.options.wrap)return;e=this.$element.find(".item")[h]()}this.sliding=!0,f&&this.pause();var j=a.Event("slide.bs.carousel",{relatedTarget:e[0],direction:g});if(!e.hasClass("active")){if(this.$indicators.length&&(this.$indicators.find(".active").removeClass("active"),this.$element.one("slid",function(){var b=a(i.$indicators.children()[i.getActiveIndex()]);b&&b.addClass("active")})),a.support.transition&&this.$element.hasClass("slide")){if(this.$element.trigger(j),j.isDefaultPrevented())return;e.addClass(b),e[0].offsetWidth,d.addClass(g),e.addClass(g),d.one(a.support.transition.end,function(){e.removeClass([b,g].join(" ")).addClass("active"),d.removeClass(["active",g].join(" ")),i.sliding=!1,se
 tTimeout(function(){i.$element.trigger("slid")},0)}).emulateTransitionEnd(600)}else{if(this.$element.trigger(j),j.isDefaultPrevented())return;d.removeClass("active"),e.addClass("active"),this.sliding=!1,this.$element.trigger("slid")}return f&&this.cycle(),this}};var c=a.fn.carousel;a.fn.carousel=function(c){return this.each(function(){var d=a(this),e=d.data("bs.carousel"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c),g="string"==typeof c?c:f.slide;e||d.data("bs.carousel",e=new b(this,f)),"number"==typeof c?e.to(c):g?e[g]():f.interval&&e.pause().cycle()})},a.fn.carousel.Constructor=b,a.fn.carousel.noConflict=function(){return a.fn.carousel=c,this},a(document).on("click.bs.carousel.data-api","[data-slide], [data-slide-to]",function(b){var c,d=a(this),e=a(d.attr("data-target")||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,"")),f=a.extend({},e.data(),d.data()),g=d.attr("data-slide-to");g&&(f.interval=!1),e.carousel(f),(g=d.attr("data-slide-to"))&&e.data("bs.carousel").to(g)
 ,b.preventDefault()}),a(window).on("load",function(){a('[data-ride="carousel"]').each(function(){var b=a(this);b.carousel(b.data())})})}(window.jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d),this.transitioning=null,this.options.parent&&(this.$parent=a(this.options.parent)),this.options.toggle&&this.toggle()};b.DEFAULTS={toggle:!0},b.prototype.dimension=function(){var a=this.$element.hasClass("width");return a?"width":"height"},b.prototype.show=function(){if(!this.transitioning&&!this.$element.hasClass("in")){var b=a.Event("show.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.$parent&&this.$parent.find("> .panel > .in");if(c&&c.length){var d=c.data("bs.collapse");if(d&&d.transitioning)return;c.collapse("hide"),d||c.data("bs.collapse",null)}var e=this.dimension();this.$element.removeClass("collapse").addClass("collapsing")[e](0),this.transitioning=1;var f=function(){this.$element.removeClass
 ("collapsing").addClass("in")[e]("auto"),this.transitioning=0,this.$element.trigger("shown.bs.collapse")};if(!a.support.transition)return f.call(this);var g=a.camelCase(["scroll",e].join("-"));this.$element.one(a.support.transition.end,a.proxy(f,this)).emulateTransitionEnd(350)[e](this.$element[0][g])}}},b.prototype.hide=function(){if(!this.transitioning&&this.$element.hasClass("in")){var b=a.Event("hide.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.dimension();this.$element[c](this.$element[c]())[0].offsetHeight,this.$element.addClass("collapsing").removeClass("collapse").removeClass("in"),this.transitioning=1;var d=function(){this.transitioning=0,this.$element.trigger("hidden.bs.collapse").removeClass("collapsing").addClass("collapse")};return a.support.transition?(this.$element[c](0).one(a.support.transition.end,a.proxy(d,this)).emulateTransitionEnd(350),void 0):d.call(this)}}},b.prototype.toggle=function(){this[this.$element.hasClass("in")?"hide":
 "show"]()};var c=a.fn.collapse;a.fn.collapse=function(c){return this.each(function(){var d=a(this),e=d.data("bs.collapse"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c);e||d.data("bs.collapse",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.collapse.Constructor=b,a.fn.collapse.noConflict=function(){return a.fn.collapse=c,this},a(document).on("click.bs.collapse.data-api","[data-toggle=collapse]",function(b){var c,d=a(this),e=d.attr("data-target")||b.preventDefault()||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,""),f=a(e),g=f.data("bs.collapse"),h=g?"toggle":d.data(),i=d.attr("data-parent"),j=i&&a(i);g&&g.transitioning||(j&&j.find('[data-toggle=collapse][data-parent="'+i+'"]').not(d).addClass("collapsed"),d[f.hasClass("in")?"addClass":"removeClass"]("collapsed")),f.collapse(h)})}(window.jQuery),+function(a){"use strict";function b(){a(d).remove(),a(e).each(function(b){var d=c(a(this));d.hasClass("open")&&(d.trigger(b=a.Event("hide.bs.dropdown")),b.isDefaultPrevented(
 )||d.removeClass("open").trigger("hidden.bs.dropdown"))})}function c(b){var c=b.attr("data-target");c||(c=b.attr("href"),c=c&&/#/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,""));var d=c&&a(c);return d&&d.length?d:b.parent()}var d=".dropdown-backdrop",e="[data-toggle=dropdown]",f=function(b){a(b).on("click.bs.dropdown",this.toggle)};f.prototype.toggle=function(d){var e=a(this);if(!e.is(".disabled, :disabled")){var f=c(e),g=f.hasClass("open");if(b(),!g){if("ontouchstart"in document.documentElement&&!f.closest(".navbar-nav").length&&a('<div class="dropdown-backdrop"/>').insertAfter(a(this)).on("click",b),f.trigger(d=a.Event("show.bs.dropdown")),d.isDefaultPrevented())return;f.toggleClass("open").trigger("shown.bs.dropdown"),e.focus()}return!1}},f.prototype.keydown=function(b){if(/(38|40|27)/.test(b.keyCode)){var d=a(this);if(b.preventDefault(),b.stopPropagation(),!d.is(".disabled, :disabled")){var f=c(d),g=f.hasClass("open");if(!g||g&&27==b.keyCode)return 27==b.which&&f.find(e).focus(),d.clic
 k();var h=a("[role=menu] li:not(.divider):visible a",f);if(h.length){var i=h.index(h.filter(":focus"));38==b.keyCode&&i>0&&i--,40==b.keyCode&&i<h.length-1&&i++,~i||(i=0),h.eq(i).focus()}}}};var g=a.fn.dropdown;a.fn.dropdown=function(b){return this.each(function(){var c=a(this),d=c.data("dropdown");d||c.data("dropdown",d=new f(this)),"string"==typeof b&&d[b].call(c)})},a.fn.dropdown.Constructor=f,a.fn.dropdown.noConflict=function(){return a.fn.dropdown=g,this},a(document).on("click.bs.dropdown.data-api",b).on("click.bs.dropdown.data-api",".dropdown form",function(a){a.stopPropagation()}).on("click.bs.dropdown.data-api",e,f.prototype.toggle).on("keydown.bs.dropdown.data-api",e+", [role=menu]",f.prototype.keydown)}(window.jQuery),+function(a){"use strict";var b=function(b,c){this.options=c,this.$element=a(b),this.$backdrop=this.isShown=null,this.options.remote&&this.$element.load(this.options.remote)};b.DEFAULTS={backdrop:!0,keyboard:!0,show:!0},b.prototype.toggle=function(a){return th
 is[this.isShown?"hide":"show"](a)},b.prototype.show=function(b){var c=this,d=a.Event("show.bs.modal",{relatedTarget:b});this.$element.trigger(d),this.isShown||d.isDefaultPrevented()||(this.isShown=!0,this.escape(),this.$element.on("click.dismiss.modal",'[data-dismiss="modal"]',a.proxy(this.hide,this)),this.backdrop(function(){var d=a.support.transition&&c.$element.hasClass("fade");c.$element.parent().length||c.$element.appendTo(document.body),c.$element.show(),d&&c.$element[0].offsetWidth,c.$element.addClass("in").attr("aria-hidden",!1),c.enforceFocus();var e=a.Event("shown.bs.modal",{relatedTarget:b});d?c.$element.find(".modal-dialog").one(a.support.transition.end,function(){c.$element.focus().trigger(e)}).emulateTransitionEnd(300):c.$element.focus().trigger(e)}))},b.prototype.hide=function(b){b&&b.preventDefault(),b=a.Event("hide.bs.modal"),this.$element.trigger(b),this.isShown&&!b.isDefaultPrevented()&&(this.isShown=!1,this.escape(),a(document).off("focusin.bs.modal"),this.$eleme
 nt.removeClass("in").attr("aria-hidden",!0).off("click.dismiss.modal"),a.support.transition&&this.$element.hasClass("fade")?this.$element.one(a.support.transition.end,a.proxy(this.hideModal,this)).emulateTransitionEnd(300):this.hideModal())},b.prototype.enforceFocus=function(){a(document).off("focusin.bs.modal").on("focusin.bs.modal",a.proxy(function(a){this.$element[0]===a.target||this.$element.has(a.target).length||this.$element.focus()},this))},b.prototype.escape=function(){this.isShown&&this.options.keyboard?this.$element.on("keyup.dismiss.bs.modal",a.proxy(function(a){27==a.which&&this.hide()},this)):this.isShown||this.$element.off("keyup.dismiss.bs.modal")},b.prototype.hideModal=function(){var a=this;this.$element.hide(),this.backdrop(function(){a.removeBackdrop(),a.$element.trigger("hidden.bs.modal")})},b.prototype.removeBackdrop=function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},b.prototype.backdrop=function(b){var c=this.$element.hasClass("fade")?"fade"
 :"";if(this.isShown&&this.options.backdrop){var d=a.support.transition&&c;if(this.$backdrop=a('<div class="modal-backdrop '+c+'" />').appendTo(document.body),this.$element.on("click.dismiss.modal",a.proxy(function(a){a.target===a.currentTarget&&("static"==this.options.backdrop?this.$element[0].focus.call(this.$element[0]):this.hide.call(this))},this)),d&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in"),!b)return;d?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()}else!this.isShown&&this.$backdrop?(this.$backdrop.removeClass("in"),a.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()):b&&b()};var c=a.fn.modal;a.fn.modal=function(c,d){return this.each(function(){var e=a(this),f=e.data("bs.modal"),g=a.extend({},b.DEFAULTS,e.data(),"object"==typeof c&&c);f||e.data("bs.modal",f=new b(this,g)),"string"==typeof c?f[c](d):g.show&&f.show(d)})},a.fn.modal.Constructor=b,a.fn.modal.
 noConflict=function(){return a.fn.modal=c,this},a(document).on("click.bs.modal.data-api",'[data-toggle="modal"]',function(b){var c=a(this),d=c.attr("href"),e=a(c.attr("data-target")||d&&d.replace(/.*(?=#[^\s]+$)/,"")),f=e.data("modal")?"toggle":a.extend({remote:!/#/.test(d)&&d},e.data(),c.data());b.preventDefault(),e.modal(f,this).one("hide",function(){c.is(":visible")&&c.focus()})}),a(document).on("show.bs.modal",".modal",function(){a(document.body).addClass("modal-open")}).on("hidden.bs.modal",".modal",function(){a(document.body).removeClass("modal-open")})}(window.jQuery),+function(a){"use strict";var b=function(a,b){this.type=this.options=this.enabled=this.timeout=this.hoverState=this.$element=null,this.init("tooltip",a,b)};b.DEFAULTS={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1},b.prototype.init=function(b,c,d){this.en
 abled=!0,this.type=b,this.$element=a(c),this.options=this.getOptions(d);for(var e=this.options.trigger.split(" "),f=e.length;f--;){var g=e[f];if("click"==g)this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this));else if("manual"!=g){var h="hover"==g?"mouseenter":"focus",i="hover"==g?"mouseleave":"blur";this.$element.on(h+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(i+"."+this.type,this.options.selector,a.proxy(this.leave,this))}}this.options.selector?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.getOptions=function(b){return b=a.extend({},this.getDefaults(),this.$element.data(),b),b.delay&&"number"==typeof b.delay&&(b.delay={show:b.delay,hide:b.delay}),b},b.prototype.getDelegateOptions=function(){var b={},c=this.getDefaults();return this._options&&a.each(this._options,function(a,d){c[a]!=d&&(b[a]=d)}),b},b.prototyp
 e.enter=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="in",c.options.delay&&c.options.delay.show?(c.timeout=setTimeout(function(){"in"==c.hoverState&&c.show()},c.options.delay.show),void 0):c.show()},b.prototype.leave=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="out",c.options.delay&&c.options.delay.hide?(c.timeout=setTimeout(function(){"out"==c.hoverState&&c.hide()},c.options.delay.hide),void 0):c.hide()},b.prototype.show=function(){var b=a.Event("show.bs."+this.type);if(this.hasContent()&&this.enabled){if(this.$element.trigger(b),b.isDefaultPrevented())return;var c=this.tip();this.setContent(),this.options.animation&&c.addClass("fade");var d="function"==typeof this.options.placement?this.options.placement.call(this,c[0],this.$ele
 ment[0]):this.options.placement,e=/\s?auto?\s?/i,f=e.test(d);f&&(d=d.replace(e,"")||"top"),c.detach().css({top:0,left:0,display:"block"}).addClass(d),this.options.container?c.appendTo(this.options.container):c.insertAfter(this.$element);var g=this.getPosition(),h=c[0].offsetWidth,i=c[0].offsetHeight;if(f){var j=this.$element.parent(),k=d,l=document.documentElement.scrollTop||document.body.scrollTop,m="body"==this.options.container?window.innerWidth:j.outerWidth(),n="body"==this.options.container?window.innerHeight:j.outerHeight(),o="body"==this.options.container?0:j.offset().left;d="bottom"==d&&g.top+g.height+i-l>n?"top":"top"==d&&g.top-l-i<0?"bottom":"right"==d&&g.right+h>m?"left":"left"==d&&g.left-h<o?"right":d,c.removeClass(k).addClass(d)}var p=this.getCalculatedOffset(d,g,h,i);this.applyPlacement(p,d),this.$element.trigger("shown.bs."+this.type)}},b.prototype.applyPlacement=function(a,b){var c,d=this.tip(),e=d[0].offsetWidth,f=d[0].offsetHeight,g=parseInt(d.css("margin-top"),10)
 ,h=parseInt(d.css("margin-left"),10);isNaN(g)&&(g=0),isNaN(h)&&(h=0),a.top=a.top+g,a.left=a.left+h,d.offset(a).addClass("in");var i=d[0].offsetWidth,j=d[0].offsetHeight;if("top"==b&&j!=f&&(c=!0,a.top=a.top+f-j),/bottom|top/.test(b)){var k=0;a.left<0&&(k=-2*a.left,a.left=0,d.offset(a),i=d[0].offsetWidth,j=d[0].offsetHeight),this.replaceArrow(k-e+i,i,"left")}else this.replaceArrow(j-f,j,"top");c&&d.offset(a)},b.prototype.replaceArrow=function(a,b,c){this.arrow().css(c,a?50*(1-a/b)+"%":"")},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle();a.find(".tooltip-inner")[this.options.html?"html":"text"](b),a.removeClass("fade in top bottom left right")},b.prototype.hide=function(){function b(){"in"!=c.hoverState&&d.detach()}var c=this,d=this.tip(),e=a.Event("hide.bs."+this.type);return this.$element.trigger(e),e.isDefaultPrevented()?void 0:(d.removeClass("in"),a.support.transition&&this.$tip.hasClass("fade")?d.one(a.support.transition.end,b).emulateTransitionEnd(150):b(),th
 is.$element.trigger("hidden.bs."+this.type),this)},b.prototype.fixTitle=function(){var a=this.$element;(a.attr("title")||"string"!=typeof a.attr("data-original-title"))&&a.attr("data-original-title",a.attr("title")||"").attr("title","")},b.prototype.hasContent=function(){return this.getTitle()},b.prototype.getPosition=function(){var b=this.$element[0];return a.extend({},"function"==typeof b.getBoundingClientRect?b.getBoundingClientRect():{width:b.offsetWidth,height:b.offsetHeight},this.$element.offset())},b.prototype.getCalculatedOffset=function(a,b,c,d){return"bottom"==a?{top:b.top+b.height,left:b.left+b.width/2-c/2}:"top"==a?{top:b.top-d,left:b.left+b.width/2-c/2}:"left"==a?{top:b.top+b.height/2-d/2,left:b.left-c}:{top:b.top+b.height/2-d/2,left:b.left+b.width}},b.prototype.getTitle=function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||("function"==typeof c.title?c.title.call(b[0]):c.title)},b.prototype.tip=function(){return this.$tip=this.$tip||a(
 this.options.template)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},b.prototype.validate=function(){this.$element[0].parentNode||(this.hide(),this.$element=null,this.options=null)},b.prototype.enable=function(){this.enabled=!0},b.prototype.disable=function(){this.enabled=!1},b.prototype.toggleEnabled=function(){this.enabled=!this.enabled},b.prototype.toggle=function(b){var c=b?a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type):this;c.tip().hasClass("in")?c.leave(c):c.enter(c)},b.prototype.destroy=function(){this.hide().$element.off("."+this.type).removeData("bs."+this.type)};var c=a.fn.tooltip;a.fn.tooltip=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tooltip"),f="object"==typeof c&&c;e||d.data("bs.tooltip",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.tooltip.Constructor=b,a.fn.tooltip.noConflict=function(){return a.fn.tooltip=c,this}}(window.jQuery),+function(a){"use strict";
 var b=function(a,b){this.init("popover",a,b)};if(!a.fn.tooltip)throw new Error("Popover requires tooltip.js");b.DEFAULTS=a.extend({},a.fn.tooltip.Constructor.DEFAULTS,{placement:"right",trigger:"click",content:"",template:'<div class="popover"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),b.prototype=a.extend({},a.fn.tooltip.Constructor.prototype),b.prototype.constructor=b,b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content")[this.options.html?"html":"text"](c),a.removeClass("fade top bottom left right in"),a.find(".popover-title").html()||a.find(".popover-title").hide()},b.prototype.hasContent=function(){return this.getTitle()||this.getContent()},b.prototype.getContent=function(){var a=this.$element,b=this.options;return a.attr("data-content")||("functio
 n"==typeof b.content?b.content.call(a[0]):b.content)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".arrow")},b.prototype.tip=function(){return this.$tip||(this.$tip=a(this.options.template)),this.$tip};var c=a.fn.popover;a.fn.popover=function(c){return this.each(function(){var d=a(this),e=d.data("bs.popover"),f="object"==typeof c&&c;e||d.data("bs.popover",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.popover.Constructor=b,a.fn.popover.noConflict=function(){return a.fn.popover=c,this}}(window.jQuery),+function(a){"use strict";function b(c,d){var e,f=a.proxy(this.process,this);this.$element=a(c).is("body")?a(window):a(c),this.$body=a("body"),this.$scrollElement=this.$element.on("scroll.bs.scroll-spy.data-api",f),this.options=a.extend({},b.DEFAULTS,d),this.selector=(this.options.target||(e=a(c).attr("href"))&&e.replace(/.*(?=#[^\s]+$)/,"")||"")+" .nav li > a",this.offsets=a([]),this.targets=a([]),this.activeTarget=null,this.refresh(),this.process(
 )}b.DEFAULTS={offset:10},b.prototype.refresh=function(){var b=this.$element[0]==window?"offset":"position";this.offsets=a([]),this.targets=a([]);var c=this;this.$body.find(this.selector).map(function(){var d=a(this),e=d.data("target")||d.attr("href"),f=/^#\w/.test(e)&&a(e);return f&&f.length&&[[f[b]().top+(!a.isWindow(c.$scrollElement.get(0))&&c.$scrollElement.scrollTop()),e]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){c.offsets.push(this[0]),c.targets.push(this[1])})},b.prototype.process=function(){var a,b=this.$scrollElement.scrollTop()+this.options.offset,c=this.$scrollElement[0].scrollHeight||this.$body[0].scrollHeight,d=c-this.$scrollElement.height(),e=this.offsets,f=this.targets,g=this.activeTarget;if(b>=d)return g!=(a=f.last()[0])&&this.activate(a);for(a=e.length;a--;)g!=f[a]&&b>=e[a]&&(!e[a+1]||b<=e[a+1])&&this.activate(f[a])},b.prototype.activate=function(b){this.activeTarget=b,a(this.selector).parents(".active").removeClass("active");var c=this.selector+
 '[data-target="'+b+'"],'+this.selector+'[href="'+b+'"]',d=a(c).parents("li").addClass("active");d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate")};var c=a.fn.scrollspy;a.fn.scrollspy=function(c){return this.each(function(){var d=a(this),e=d.data("bs.scrollspy"),f="object"==typeof c&&c;e||d.data("bs.scrollspy",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=c,this},a(window).on("load",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);b.scrollspy(b.data())})})}(window.jQuery),+function(a){"use strict";var b=function(b){this.element=a(b)};b.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.attr("data-target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a")[0],f=a.Event("show.bs.tab",{relatedTarget:e});if(b.trigger(f
 ),!f.isDefaultPrevented()){var g=a(d);this.activate(b.parent("li"),c),this.activate(g,g.parent(),function(){b.trigger({type:"shown.bs.tab",relatedTarget:e})})}}},b.prototype.activate=function(b,c,d){function e(){f.removeClass("active").find("> .dropdown-menu > .active").removeClass("active"),b.addClass("active"),g?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu")&&b.closest("li.dropdown").addClass("active"),d&&d()}var f=c.find("> .active"),g=d&&a.support.transition&&f.hasClass("fade");g?f.one(a.support.transition.end,e).emulateTransitionEnd(150):e(),f.removeClass("in")};var c=a.fn.tab;a.fn.tab=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new b(this)),"string"==typeof c&&e[c]()})},a.fn.tab.Constructor=b,a.fn.tab.noConflict=function(){return a.fn.tab=c,this},a(document).on("click.bs.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(b){b.preventDefault(),a(this).tab("show")})}(window.jQ
 uery),+function(a){"use strict";var b=function(c,d){this.options=a.extend({},b.DEFAULTS,d),this.$window=a(window).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(c),this.affixed=this.unpin=null,this.checkPosition()};b.RESET="affix affix-top affix-bottom",b.DEFAULTS={offset:0},b.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},b.prototype.checkPosition=function(){if(this.$element.is(":visible")){var c=a(document).height(),d=this.$window.scrollTop(),e=this.$element.offset(),f=this.options.offset,g=f.top,h=f.bottom;"object"!=typeof f&&(h=g=f),"function"==typeof g&&(g=f.top()),"function"==typeof h&&(h=f.bottom());var i=null!=this.unpin&&d+this.unpin<=e.top?!1:null!=h&&e.top+this.$element.height()>=c-h?"bottom":null!=g&&g>=d?"top":!1;this.affixed!==i&&(this.unpin&&this.$element.css("top",""),this.affixed=i,this.unpin="bottom"==i?e.top-
 d:null,this.$element.removeClass(b.RESET).addClass("affix"+(i?"-"+i:"")),"bottom"==i&&this.$element.offset({top:document.body.offsetHeight-h-this.$element.height()}))}};var c=a.fn.affix;a.fn.affix=function(c){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof c&&c;e||d.data("bs.affix",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.affix.Constructor=b,a.fn.affix.noConflict=function(){return a.fn.affix=c,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var b=a(this),c=b.data();c.offset=c.offset||{},c.offsetBottom&&(c.offset.bottom=c.offsetBottom),c.offsetTop&&(c.offset.top=c.offsetTop),b.affix(c)})})}(window.jQuery);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/js/head.js
----------------------------------------------------------------------
diff --git a/gemfire-site/content/js/head.js b/gemfire-site/content/js/head.js
deleted file mode 100644
index aa15cb8..0000000
--- a/gemfire-site/content/js/head.js
+++ /dev/null
@@ -1,708 +0,0 @@
-// // the most simple case. load and execute single script without blocking.
-// head.js("/path/to/file.js");
- 
-// // load a script and execute a function after it has been loaded
-// head.js("/path/to/file.js", function() {
- 
-// });
- 
-// // load files in parallel but execute them in sequence
-// head.js("file1.js", "file2.js", ... "fileN.js");
- 
-// // execute function after all scripts have been loaded
-// head.js("file1.js", "file2.js", function() {
- 
-// });
- 
-// // files are loaded in parallel and executed in order they arrive
-// head.js("file1.js");
-// head.js("file2.js");
-// head.js("file3.js");
- 
-// // the previous can also be written as
-// head.js("file1.js").js("file1.js").js("file3.js");
-
-
-
-
-
-/**
-    Head JS     The only script in your <HEAD>
-    Copyright   Tero Piirainen (tipiirai)
-    License     MIT / http://bit.ly/mit-license
-    Version     0.96
-
-    http://headjs.com
-*/
-(function(doc) {
-
-    var html = doc.documentElement,
-         conf = {
-            screens: [320, 480, 640, 768, 1024, 1280, 1440, 1680, 1920],
-            section: "-section",
-            page: "-page",
-            head: "head"
-         },
-         klass = [];
-
-
-    if (window.head_conf) {
-        for (var key in head_conf) {
-            if (head_conf[key] !== undefined) {
-                conf[key] = head_conf[key];
-            }
-        }
-    }
-
-    function pushClass(name) {
-        klass[klass.length] = name;
-    }
-
-    function removeClass(name) {
-        var re = new RegExp("\\b" + name + "\\b");
-        html.className = html.className.replace(re, '');
-    }
-
-    function each(arr, fn) {
-        for (var i = 0, arr_length = arr.length; i < arr_length; i++) {
-            fn.call(arr, arr[i], i);
-        }
-    }
-
-    // API
-    var api = window[conf.head] = function() {
-        api.ready.apply(null, arguments);
-    };
-
-    api.feature = function(key, enabled, queue) {
-
-        // internal: apply all classes
-        if (!key) {
-            html.className += ' ' + klass.join( ' ' );
-            klass = [];
-            return;
-        }
-
-        if (Object.prototype.toString.call(enabled) == '[object Function]') {
-            enabled = enabled.call();
-        }
-
-        pushClass((enabled ? '' : 'no-') + key);
-        api[key] = !!enabled;
-
-        // apply class to HTML element
-        if (!queue) {
-            removeClass('no-' + key);
-            removeClass(key);
-            api.feature();
-        }
-
-        return api;
-    };
-
-    // browser type & version
-    var ua = navigator.userAgent.toLowerCase();
-
-    ua = /(webkit)[ \/]([\w.]+)/.exec( ua ) ||
-        /(opera)(?:.*version)?[ \/]([\w.]+)/.exec( ua ) ||
-        /(msie) ([\w.]+)/.exec( ua ) ||
-        !/compatible/.test( ua ) && /(mozilla)(?:.*? rv:([\w.]+))?/.exec( ua ) || [];
-
-
-    if (ua[1] == 'msie') {
-        ua[1] = 'ie';
-        ua[2] = document.documentMode || ua[2];
-    }
-
-    pushClass(ua[1]);
-
-    api.browser = { version: ua[2] };
-    api.browser[ua[1]] = true;
-
-    // IE specific
-    if (api.browser.ie) {
-
-        pushClass("ie" + parseFloat(ua[2]));
-
-        // IE versions
-        for (var ver = 3; ver < 11; ver++) {
-            if (parseFloat(ua[2]) < ver) { pushClass("lt-ie" + ver); }
-        }
-
-        // HTML5 support
-        each("abbr|article|aside|audio|canvas|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video".split("|"), function(el) {
-            doc.createElement(el);
-        });
-                
-    }
-    
-
-    // CSS "router"
-    each(location.pathname.split("/"), function(el, i) {
-
-        if (this.length > 2 && this[i + 1] !== undefined) {
-            if (i) { pushClass(this.slice(1, i+1).join("-") + conf.section); }
-
-        } else {
-
-            // pageId
-            var id = el || "index", index = id.indexOf(".");
-            if (index > 0) { id = id.substring(0, index); }
-            html.id = id + conf.page;
-
-            // on root?
-            if (!i) { pushClass("root" + conf.section); }
-      }
-    });
-
-
-    // screen resolution: w-100, lt-480, lt-1024 ...
-    function screenSize() {
-        var w = window.outerWidth || html.clientWidth;
-
-        // remove earlier widths
-        html.className = html.className.replace(/ (w|lt)-\d+/g, "");
-
-        // add new ones
-        pushClass("w-" + Math.round(w / 100) * 100);
-
-        each(conf.screens, function(width) {
-            if (w <= width) { pushClass("lt-" + width); }
-        });
-
-        api.feature();
-    }
-
-    screenSize();
-    window.onresize = screenSize;
-
-    api.feature("js", true).feature();
-
-})(document);
-
-
-/**
-    Head JS     The only script in your <HEAD>
-    Copyright   Tero Piirainen (tipiirai)
-    License     MIT / http://bit.ly/mit-license
-    Version     0.96
-
-    http://headjs.com
-*/
-(function() {
-    /*
-        To add a new test:
-
-        head.feature("video", function() {
-            var tag = document.createElement('video');
-            return !!tag.canPlayType;
-        });
-
-        Good place to grab more tests
-
-        https://github.com/Modernizr/Modernizr/blob/master/modernizr.js
-    */
-
-
-    /* CSS modernizer */
-    var el = document.createElement("i"),
-         style = el.style,
-         prefs = ' -o- -moz- -ms- -webkit- -khtml- '.split(' '),
-         domPrefs = 'Webkit Moz O ms Khtml'.split(' '),
-
-         head_var = window.head_conf && head_conf.head || "head",
-         api = window[head_var];
-
-
-     // Thanks Paul Irish!
-    function testProps(props) {
-        for (var i in props) {
-            if (style[props[i]] !== undefined) {
-                return true;
-            }
-        }
-    }
-
-
-    function testAll(prop) {
-        var camel = prop.charAt(0).toUpperCase() + prop.substr(1),
-             props   = (prop + ' ' + domPrefs.join(camel + ' ') + camel).split(' ');
-
-        return !!testProps(props);
-    }
-
-    var tests = {
-
-        gradient: function() {
-            var s1 = 'background-image:',
-                 s2 = 'gradient(linear,left top,right bottom,from(#9f9),to(#fff));',
-                 s3 = 'linear-gradient(left top,#eee,#fff);';
-
-            style.cssText = (s1 + prefs.join(s2 + s1) + prefs.join(s3 + s1)).slice(0,-s1.length);
-            return !!style.backgroundImage;
-        },
-
-        rgba: function() {
-            style.cssText = "background-color:rgba(0,0,0,0.5)";
-            return !!style.backgroundColor;
-        },
-
-        opacity: function() {
-            return el.style.opacity === "";
-        },
-
-        textshadow: function() {
-            return style.textShadow === '';
-        },
-
-        multiplebgs: function() {
-            style.cssText = "background:url(//:),url(//:),red url(//:)";
-            return new RegExp("(url\\s*\\(.*?){3}").test(style.background);
-        },
-
-        boxshadow: function() {
-            return testAll("boxShadow");
-        },
-
-        borderimage: function() {
-            return testAll("borderImage");
-        },
-
-        borderradius: function() {
-            return testAll("borderRadius");
-        },
-
-        cssreflections: function() {
-            return testAll("boxReflect");
-        },
-
-        csstransforms: function() {
-            return testAll("transform");
-        },
-
-        csstransitions: function() {
-            return testAll("transition");
-        },
-
-        /*
-            font-face support. Uses browser sniffing but is synchronous.
-
-            http://paulirish.com/2009/font-face-feature-detection/
-        */
-        fontface: function() {
-            var ua = navigator.userAgent, parsed;
-
-            if (/*@cc_on@if(@_jscript_version>=5)!@end@*/0)
-                return true;
-                
-            if (parsed = ua.match(/Chrome\/(\d+\.\d+\.\d+\.\d+)/))
-                return parsed[1] >= '4.0.249.4' || 1 * parsed[1].split(".")[0] > 5;
-            if ((parsed = ua.match(/Safari\/(\d+\.\d+)/)) && !/iPhone/.test(ua))
-                return parsed[1] >= '525.13';
-            if (/Opera/.test({}.toString.call(window.opera)))
-                return opera.version() >= '10.00';
-            if (parsed = ua.match(/rv:(\d+\.\d+\.\d+)[^b].*Gecko\//))
-                return parsed[1] >= '1.9.1';
-
-            return false;
-        }
-    };
-
-    // queue features
-    for (var key in tests) {
-        if (tests[key]) {
-            api.feature(key, tests[key].call(), true);
-        }
-    }
-
-    // enable features at once
-    api.feature();
-
-})();
-
-
-/**
-    Head JS     The only script in your <HEAD>
-    Copyright   Tero Piirainen (tipiirai)
-    License     MIT / http://bit.ly/mit-license
-    Version     0.96
-
-    http://headjs.com
-*/
-(function(doc) {
-
-    var head = doc.documentElement,
-        isHeadReady,
-        isDomReady,
-        domWaiters = [],
-        queue = [],        // waiters for the "head ready" event
-        handlers = {},     // user functions waiting for events
-        scripts = {},      // loadable scripts in different states
-        isAsync = doc.createElement("script").async === true || "MozAppearance" in doc.documentElement.style || window.opera;
-
-
-    /*** public API ***/
-    var head_var = window.head_conf && head_conf.head || "head",
-         api = window[head_var] = (window[head_var] || function() { api.ready.apply(null, arguments); });
-
-    // states
-    var PRELOADED = 1,
-        PRELOADING = 2,
-        LOADING = 3,
-        LOADED = 4;
-
-
-    // Method 1: simply load and let browser take care of ordering
-    if (isAsync) {
-
-        api.js = function() {
-
-            var args = arguments,
-                 fn = args[args.length -1],
-                 els = {};
-
-            if (!isFunc(fn)) { fn = null; }
-
-            each(args, function(el, i) {
-
-                if (el != fn) {
-                    el = getScript(el);
-                    els[el.name] = el;
-
-                    load(el, fn && i == args.length -2 ? function() {
-                        if (allLoaded(els)) { one(fn); }
-
-                    } : null);
-                }
-            });
-
-            return api;
-        };
-
-
-    // Method 2: preload with text/cache hack
-    } else {
-
-        api.js = function() {
-
-            var args = arguments,
-                rest = [].slice.call(args, 1),
-                next = rest[0];
-
-            // wait for a while. immediate execution causes some browsers to ignore caching
-            if (!isHeadReady) {
-                queue.push(function()  {
-                    api.js.apply(null, args);
-                });
-                return api;
-            }
-
-            // multiple arguments
-            if (next) {
-
-                // load
-                each(rest, function(el) {
-                    if (!isFunc(el)) {
-                        preload(getScript(el));
-                    }
-                });
-
-                // execute
-                load(getScript(args[0]), isFunc(next) ? next : function() {
-                    api.js.apply(null, rest);
-                });
-
-
-            // single script
-            } else {
-                load(getScript(args[0]));
-            }
-
-            return api;
-        };
-    }
-
-    api.ready = function(key, fn) {
-
-        // DOM ready check: head.ready(document, function() { });
-        if (key == doc) {
-            if (isDomReady) { one(fn);  }
-            else { domWaiters.push(fn); }
-            return api;
-        }
-
-        // shift arguments
-        if (isFunc(key)) {
-            fn = key;
-            key = "ALL";
-        }    
-
-        // make sure arguments are sane
-        if (typeof key != 'string' || !isFunc(fn)) { return api; }
-
-        var script = scripts[key];
-        
-        // script already loaded --> execute and return
-        if (script && script.state == LOADED || key == 'ALL' && allLoaded() && isDomReady) {
-            one(fn);
-            return api;
-        }
-
-        var arr = handlers[key];
-        if (!arr) { arr = handlers[key] = [fn]; }
-        else { arr.push(fn); }
-        return api;
-    };
-
-
-    // perform this when DOM is ready
-    api.ready(doc, function() {
-
-        if (allLoaded()) {
-            each(handlers.ALL, function(fn) {
-                one(fn);
-            });
-        }
-
-        if (api.feature) {
-            api.feature("domloaded", true);
-        }
-    });
-
-
-    /*** private functions ***/
-    
-    
-    // call function once
-    function one(fn) {
-        if (fn._done) { return; }
-        fn();
-        fn._done = 1;
-    }
-
-
-    function toLabel(url) {
-        var els = url.split("/"),
-             name = els[els.length -1],
-             i = name.indexOf("?");
-
-        return i != -1 ? name.substring(0, i) : name;
-    }
-
-
-    function getScript(url) {
-
-        var script;
-
-        if (typeof url == 'object') {
-            for (var key in url) {
-                if (url[key]) {
-                    script = { name: key, url: url[key] };
-                }
-            }
-        } else {
-            script = { name: toLabel(url),  url: url };
-        }
-
-        var existing = scripts[script.name];
-        if (existing && existing.url === script.url) { return existing; }
-
-        scripts[script.name] = script;
-        return script;
-    }
-
-
-    function each(arr, fn) {
-        if (!arr) { return; }
-
-        // arguments special type
-        if (typeof arr == 'object') { arr = [].slice.call(arr); }
-
-        // do the job
-        for (var i = 0; i < arr.length; i++) {
-            fn.call(arr, arr[i], i);
-        }
-    }
-
-    function isFunc(el) {
-        return Object.prototype.toString.call(el) == '[object Function]';
-    }
-
-    function allLoaded(els) {
-
-        els = els || scripts;
-
-        var loaded;
-        
-        for (var name in els) {
-            if (els.hasOwnProperty(name) && els[name].state != LOADED) { return false; }
-            loaded = true;
-        }
-        
-        return loaded;
-    }
-
-
-    function onPreload(script) {
-        script.state = PRELOADED;
-
-        each(script.onpreload, function(el) {
-            el.call();
-        });
-    }
-
-    function preload(script, callback) {
-
-        if (script.state === undefined) {
-
-            script.state = PRELOADING;
-            script.onpreload = [];
-
-            scriptTag({ src: script.url, type: 'cache'}, function()  {
-                onPreload(script);
-            });
-        }
-    }
-
-    function load(script, callback) {
-
-        if (script.state == LOADED) {
-            return callback && callback();
-        }
-
-        if (script.state == LOADING) {
-            return api.ready(script.name, callback);
-        }
-
-        if (script.state == PRELOADING) {
-            return script.onpreload.push(function() {
-                load(script, callback);
-            });
-        }
-
-        script.state = LOADING;
-
-        scriptTag(script.url, function() {
-
-            script.state = LOADED;
-
-            if (callback) { callback(); }
-
-            // handlers for this script
-            each(handlers[script.name], function(fn) {
-                one(fn);
-            });
-
-            // everything ready
-            if (allLoaded() && isDomReady) {
-                each(handlers.ALL, function(fn) {
-                    one(fn);
-                });
-            }
-        });
-    }
-
-
-    function scriptTag(src, callback) {
-
-        var s = doc.createElement('script');
-        s.type = 'text/' + (src.type || 'javascript');
-        s.src = src.src || src;
-        s.async = false;
-
-        s.onreadystatechange = s.onload = function() {
-
-            var state = s.readyState;
-
-            if (!callback.done && (!state || /loaded|complete/.test(state))) {
-                callback.done = true;
-                callback();
-            }
-        };
-
-        // use body if available. more safe in IE
-        (doc.body || head).appendChild(s);
-    }
-
-    /*
-        The much desired DOM ready check
-        Thanks to jQuery and http://javascript.nwbox.com/IEContentLoaded/
-    */
-
-    function fireReady() {
-        if (!isDomReady) {
-            isDomReady = true;
-            each(domWaiters, function(fn) {
-                one(fn);
-            });
-        }
-    }
-
-    // W3C
-    if (window.addEventListener) {
-        doc.addEventListener("DOMContentLoaded", fireReady, false);
-
-        // fallback. this is always called
-        window.addEventListener("load", fireReady, false);
-
-    // IE
-    } else if (window.attachEvent) {
-
-        // for iframes
-        doc.attachEvent("onreadystatechange", function()  {
-            if (doc.readyState === "complete" ) {
-                fireReady();
-            }
-        });
-
-
-        // avoid frames with different domains issue
-        var frameElement = 1;
-
-        try {
-            frameElement = window.frameElement;
-
-        } catch(e) {}
-
-
-        if (!frameElement && head.doScroll) {
-
-            (function() {
-                try {
-                    head.doScroll("left");
-                    fireReady();
-
-                } catch(e) {
-                    setTimeout(arguments.callee, 1);
-                    return;
-                }
-            })();
-        }
-
-        // fallback
-        window.attachEvent("onload", fireReady);
-    }
-
-
-    // enable document.readyState for Firefox <= 3.5
-    if (!doc.readyState && doc.addEventListener) {
-        doc.readyState = "loading";
-        doc.addEventListener("DOMContentLoaded", handler = function () {
-            doc.removeEventListener("DOMContentLoaded", handler, false);
-            doc.readyState = "complete";
-        }, false);
-    }
-
-    /*
-        We wait for 300 ms before script loading starts. for some reason this is needed
-        to make sure scripts are cached. Not sure why this happens yet. A case study:
-
-        https://github.com/headjs/headjs/issues/closed#issue/83
-    */
-    setTimeout(function() {
-        isHeadReady = true;
-        each(queue, function(fn) { fn(); });
-
-    }, 300);
-
-})(document);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/js/html5shiv.js
----------------------------------------------------------------------
diff --git a/gemfire-site/content/js/html5shiv.js b/gemfire-site/content/js/html5shiv.js
deleted file mode 100644
index 784f221..0000000
--- a/gemfire-site/content/js/html5shiv.js
+++ /dev/null
@@ -1,8 +0,0 @@
-/*
- HTML5 Shiv v3.6.2pre | @afarkas @jdalton @jon_neal @rem | MIT/GPL2 Licensed
-*/
-(function(l,f){function m(){var a=e.elements;return"string"==typeof a?a.split(" "):a}function i(a){var b=n[a[o]];b||(b={},h++,a[o]=h,n[h]=b);return b}function p(a,b,c){b||(b=f);if(g)return b.createElement(a);c||(c=i(b));b=c.cache[a]?c.cache[a].cloneNode():r.test(a)?(c.cache[a]=c.createElem(a)).cloneNode():c.createElem(a);return b.canHaveChildren&&!s.test(a)?c.frag.appendChild(b):b}function t(a,b){if(!b.cache)b.cache={},b.createElem=a.createElement,b.createFrag=a.createDocumentFragment,b.frag=b.createFrag();
-a.createElement=function(c){return!e.shivMethods?b.createElem(c):p(c,a,b)};a.createDocumentFragment=Function("h,f","return function(){var n=f.cloneNode(),c=n.createElement;h.shivMethods&&("+m().join().replace(/\w+/g,function(a){b.createElem(a);b.frag.createElement(a);return'c("'+a+'")'})+");return n}")(e,b.frag)}function q(a){a||(a=f);var b=i(a);if(e.shivCSS&&!j&&!b.hasCSS){var c,d=a;c=d.createElement("p");d=d.getElementsByTagName("head")[0]||d.documentElement;c.innerHTML="x<style>article,aside,figcaption,figure,footer,header,hgroup,nav,section{display:block}mark{background:#FF0;color:#000}</style>";
-c=d.insertBefore(c.lastChild,d.firstChild);b.hasCSS=!!c}g||t(a,b);return a}var k=l.html5||{},s=/^<|^(?:button|map|select|textarea|object|iframe|option|optgroup)$/i,r=/^(?:a|b|code|div|fieldset|h1|h2|h3|h4|h5|h6|i|label|li|ol|p|q|span|strong|style|table|tbody|td|th|tr|ul)$/i,j,o="_html5shiv",h=0,n={},g;(function(){try{var a=f.createElement("a");a.innerHTML="<xyz></xyz>";j="hidden"in a;var b;if(!(b=1==a.childNodes.length)){f.createElement("a");var c=f.createDocumentFragment();b="undefined"==typeof c.cloneNode||
-"undefined"==typeof c.createDocumentFragment||"undefined"==typeof c.createElement}g=b}catch(d){g=j=!0}})();var e={elements:k.elements||"abbr article aside audio bdi canvas data datalist details figcaption figure footer header hgroup mark meter nav output progress section summary time video",version:"3.6.2pre",shivCSS:!1!==k.shivCSS,supportsUnknownElements:g,shivMethods:!1!==k.shivMethods,type:"default",shivDocument:q,createElement:p,createDocumentFragment:function(a,b){a||(a=f);if(g)return a.createDocumentFragment();
-for(var b=b||i(a),c=b.frag.cloneNode(),d=0,e=m(),h=e.length;d<h;d++)c.createElement(e[d]);return c}};l.html5=e;q(f)})(this,document);


[02/50] [abbrv] incubator-geode git commit: GEODE-624: add unit test for LifeCycleListener

Posted by je...@apache.org.
GEODE-624: add unit test for LifeCycleListener

Renamed SimpleMemoryAllocatorLifeCycleListener to LifeCycleListener.
New methods in RefCountChangeInfo for unit testing.
Renamed ReferenceCountHelper method isDuplicate to isSameCaller.

This closes #53


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

Branch: refs/heads/feature/GEODE-14
Commit: 412f2415ea6f5fb20346e3bd722d5defc0b606a3
Parents: 9d047b4
Author: Scott Jewell <sj...@pivotal.io>
Authored: Thu Dec 10 10:30:18 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Fri Dec 11 13:35:55 2015 -0800

----------------------------------------------------------------------
 .../internal/offheap/RefCountChangeInfo.java    |  77 ++++---
 .../internal/offheap/ReferenceCountHelper.java  |  14 +-
 .../offheap/LifecycleListenerJUnitTest.java     | 222 +++++++++++++++++++
 .../offheap/RefCountChangeInfoJUnitTest.java    | 140 ++++++++----
 ...moryAllocatorLifecycleListenerJUnitTest.java | 147 ------------
 5 files changed, 367 insertions(+), 233 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/412f2415/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
index 67688ed..e3b4b1f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfo.java
@@ -29,7 +29,7 @@ public class RefCountChangeInfo extends Throwable {
   private final String threadName;
   private final int rc;
   private final Object owner;
-  private int dupCount;
+  private int useCount;
   
   public RefCountChangeInfo(boolean decRefCount, int rc, Object owner) {
     super(decRefCount ? "FREE" : "USED");
@@ -42,11 +42,16 @@ public class RefCountChangeInfo extends Throwable {
     return this.owner;
   }
   
-  public int getDupCount() {
-    return this.dupCount;
+  public int getUseCount() {
+    return this.useCount;
   }
-  public void decDupCount() {
-    this.dupCount--;
+  public int incUseCount() {
+    this.useCount++;
+    return this.useCount;
+  }  
+  public int decUseCount() {
+    this.useCount--;
+    return this.useCount;
   }
 
   @Override
@@ -56,9 +61,9 @@ public class RefCountChangeInfo extends Throwable {
     ps.print(this.getMessage());
     ps.print(" rc=");
     ps.print(this.rc);
-    if (this.dupCount > 0) {
-      ps.print(" dupCount=");
-      ps.print(this.dupCount);
+    if (this.useCount > 0) {
+      ps.print(" useCount=");
+      ps.print(this.useCount);
     }
     ps.print(" by ");
     ps.print(this.threadName);
@@ -75,47 +80,51 @@ public class RefCountChangeInfo extends Throwable {
     
     return baos.toString();
   }
-  
-  public boolean isDuplicate(RefCountChangeInfo other) {
+
+  public boolean isSameCaller(RefCountChangeInfo other) {
     if (!getMessage().equals(other.getMessage())) return false;
-    String trace = getStackTraceString();
-    String traceOther = other.getStackTraceString();
+    Object trace = getStackTraceString();
+    Object traceOther = other.getStackTraceString();
     if (trace.hashCode() != traceOther.hashCode()) return false;
     if (trace.equals(traceOther)) {
-      this.dupCount++;
       return true;
     } else {
       return false;
     }
   }
 
-  private String stackTraceString;
-  String getStackTraceString() {
-    String result = this.stackTraceString;
-    if (result == null) {
-	ByteArrayOutputStream baos = new ByteArrayOutputStream(64*1024);
-	PrintStream spr = new PrintStream(baos);
+  private Object stackTraceString;
 
-	cleanStackTrace(spr);
-	result = baos.toString();
-	this.stackTraceString = result;
+  Object getStackTraceString() {
+    Object result = this.stackTraceString;
+    if (result == null) {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream(64 * 1024);
+      PrintStream spr = new PrintStream(baos);
+      cleanStackTrace(spr);
+      result = baos.toString();
+      this.stackTraceString = result;
     }
     return result;
   }
   
+  void setStackTraceString(Object sts) {
+    stackTraceString = sts;
+  }
+
   private void cleanStackTrace(PrintStream ps) {
-      StackTraceElement[] trace = getStackTrace();
-      // skip the initial elements from the offheap package
-      int skip=0;
-      for (int i=0; i < trace.length; i++) {
-	if (!trace[i].getClassName().contains("com.gemstone.gemfire.internal.offheap")) {
-          skip = i;
-          break;
-        }
+    StackTraceElement[] trace = getStackTrace();
+    // skip the initial elements from the offheap package
+    int skip=0;
+    for (int i=0; i < trace.length; i++) {
+      if(!(trace[i].toString().contains("com.gemstone.gemfire.internal.offheap"))) {
+        skip = i;
+        break;
       }
-      for (int i=skip; i < trace.length; i++) {
-        ps.println("\tat " + trace[i]);
-      }   
-  }
+    }
+    for (int i=skip; i < trace.length; i++) {
+      ps.println("\tat " + trace[i]);
+    }   
+}
+
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/412f2415/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelper.java
index e396060..778b329 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelper.java
@@ -170,6 +170,7 @@ public class ReferenceCountHelper {
   /**
    * Used internally to report that a reference count has changed.
    */
+  
   static void refCountChanged(Long address, boolean decRefCount, int rc) {
     final Object owner = refCountOwner.get();
     if (owner == SKIP_REF_COUNT_TRACKING) {
@@ -193,16 +194,16 @@ public class ReferenceCountHelper {
             if (owner instanceof RegionEntry) {
               // use identity comparison on region entries since sqlf does some wierd stuff in the equals method
               if (owner == info.getOwner()) {
-                if (info.getDupCount() > 0) {
-                  info.decDupCount();
+                if (info.getUseCount() > 0) {
+                  info.decUseCount();
                 } else {
                   list.remove(i);
                 }
                 return;
               }
             } else if (owner.equals(info.getOwner())) {
-              if (info.getDupCount() > 0) {
-                info.decDupCount();
+              if (info.getUseCount() > 0) {
+                info.decUseCount();
               } else {
                 list.remove(i);
               }
@@ -223,8 +224,9 @@ public class ReferenceCountHelper {
       //        list.clear();
       //      }
       for (RefCountChangeInfo e: list) {
-        if (e.isDuplicate(info)) {
-          // No need to add it
+        if (e.isSameCaller(info)) {
+          // No need to add it just increment useCount
+          e.incUseCount();
           return;
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/412f2415/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
new file mode 100755
index 0000000..c886e43
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/LifecycleListenerJUnitTest.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Tests SimpleMemoryAllocatorImpl.LifecycleListener
+ * 
+ * @author Kirk Lund
+ */
+@Category(UnitTest.class)
+public class LifecycleListenerJUnitTest {
+
+  private final List<LifecycleListenerCallback> afterCreateCallbacks = new ArrayList<LifecycleListenerCallback>();
+  private final List<LifecycleListenerCallback> afterReuseCallbacks = new ArrayList<LifecycleListenerCallback>();
+  private final List<LifecycleListenerCallback> beforeCloseCallbacks = new ArrayList<LifecycleListenerCallback>();
+  private final TestLifecycleListener listener = new TestLifecycleListener(this.afterCreateCallbacks, this.afterReuseCallbacks, this.beforeCloseCallbacks);
+
+  @After
+  public void tearDown() throws Exception {
+    LifecycleListener.removeLifecycleListener(this.listener);
+    this.afterCreateCallbacks.clear();
+    this.afterReuseCallbacks.clear();
+    this.beforeCloseCallbacks.clear();
+    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  @Test
+  public void testAddRemoveListener() {
+    LifecycleListener.addLifecycleListener(this.listener);
+    LifecycleListener.removeLifecycleListener(this.listener);
+
+    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
+    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(),
+        new UnsafeMemoryChunk[] { slab });
+
+    Assert.assertEquals(0, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(0, this.beforeCloseCallbacks.size());
+
+    ma.close();
+
+    Assert.assertEquals(0, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(0, this.beforeCloseCallbacks.size());
+
+    LifecycleListener.removeLifecycleListener(this.listener);
+  }
+
+  @Test
+  public void testCallbacksAreCalledAfterCreate() {
+    LifecycleListener.addLifecycleListener(this.listener);
+    // saj
+    System.getProperties().put("gemfire.free-off-heap-memory", "true");
+    // saj above
+    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
+    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(),
+        new UnsafeMemoryChunk[] { slab });
+
+    Assert.assertEquals(1, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(0, this.beforeCloseCallbacks.size());
+
+    ma.close();
+
+    Assert.assertEquals(1, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(1, this.beforeCloseCallbacks.size());
+    
+    LifecycleListener.removeLifecycleListener(this.listener);
+  }
+
+  @Test
+  public void testCallbacksAreCalledAfterReuse() {
+
+    LifecycleListener.addLifecycleListener(this.listener);
+
+    System.getProperties().put("gemfire.free-off-heap-memory", "false");
+
+    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
+    SimpleMemoryAllocatorImpl ma = createAllocator(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[] { slab });
+
+    Assert.assertEquals(1, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(0, this.beforeCloseCallbacks.size());
+
+    ma.close();
+
+    Assert.assertEquals(1, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(1, this.beforeCloseCallbacks.size());
+
+    ma = createAllocator(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), null);
+
+    Assert.assertEquals(1, this.afterCreateCallbacks.size());
+    Assert.assertEquals(1, this.afterReuseCallbacks.size());
+    Assert.assertEquals(1, this.beforeCloseCallbacks.size());
+
+    SimpleMemoryAllocatorImpl ma2 = createAllocator(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[] { slab });
+    assertEquals(null, ma2);
+    
+    Assert.assertEquals(1, this.afterCreateCallbacks.size());
+    Assert.assertEquals(1, this.afterReuseCallbacks.size());
+    Assert.assertEquals(1, this.beforeCloseCallbacks.size());
+
+    ma.close();
+
+    Assert.assertEquals(1, this.afterCreateCallbacks.size());
+    Assert.assertEquals(1, this.afterReuseCallbacks.size());
+    Assert.assertEquals(2, this.beforeCloseCallbacks.size());
+  }
+
+  private SimpleMemoryAllocatorImpl createAllocator(OutOfOffHeapMemoryListener ooohml, OffHeapMemoryStats ohms, UnsafeMemoryChunk[] slab) {
+    try {
+       return SimpleMemoryAllocatorImpl.create(ooohml, ohms, slab);
+    } catch (IllegalStateException e) {
+      return null;
+    }
+  }
+  
+  @Test
+  public void testCallbacksAreCalledAfterReuseWithFreeTrue() {
+
+    LifecycleListener.addLifecycleListener(this.listener);
+
+    System.getProperties().put("gemfire.free-off-heap-memory", "true");
+
+    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
+    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[] { slab });
+
+    Assert.assertEquals(1, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(0, this.beforeCloseCallbacks.size());
+
+    ma.close();
+
+    Assert.assertEquals(1, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(1, this.beforeCloseCallbacks.size());
+
+    slab = new UnsafeMemoryChunk(1024); // 1k
+    SimpleMemoryAllocatorImpl ma2 = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[] { slab });
+
+    Assert.assertEquals(2, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(1, this.beforeCloseCallbacks.size());
+
+    ma.close();
+
+    Assert.assertEquals(2, this.afterCreateCallbacks.size());
+    Assert.assertEquals(0, this.afterReuseCallbacks.size());
+    Assert.assertEquals(2, this.beforeCloseCallbacks.size());
+  }
+
+  static final class LifecycleListenerCallback {
+    private final SimpleMemoryAllocatorImpl allocator;
+    private final long timeStamp;
+    private final Throwable creationTime;
+
+    LifecycleListenerCallback(SimpleMemoryAllocatorImpl allocator) {
+      this.allocator = allocator;
+      this.timeStamp = System.currentTimeMillis();
+      this.creationTime = new Exception();
+    }
+  }
+
+  static class TestLifecycleListener implements LifecycleListener {
+    private final List<LifecycleListenerCallback> afterCreateCallbacks;
+    private final List<LifecycleListenerCallback> afterReuseCallbacks;
+    private final List<LifecycleListenerCallback> beforeCloseCallbacks;
+
+    TestLifecycleListener(List<LifecycleListenerCallback> afterCreateCallbacks, List<LifecycleListenerCallback> afterReuseCallbacks,
+        List<LifecycleListenerCallback> beforeCloseCallbacks) {
+      this.afterCreateCallbacks = afterCreateCallbacks;
+      this.afterReuseCallbacks = afterReuseCallbacks;
+      this.beforeCloseCallbacks = beforeCloseCallbacks;
+    }
+
+    @Override
+    public void afterCreate(SimpleMemoryAllocatorImpl allocator) {
+      this.afterCreateCallbacks.add(new LifecycleListenerCallback(allocator));
+    }
+
+    @Override
+    public void afterReuse(SimpleMemoryAllocatorImpl allocator) {
+      this.afterReuseCallbacks.add(new LifecycleListenerCallback(allocator));
+    }
+
+    @Override
+    public void beforeClose(SimpleMemoryAllocatorImpl allocator) {
+      this.beforeCloseCallbacks.add(new LifecycleListenerCallback(allocator));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/412f2415/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfoJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfoJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfoJUnitTest.java
index fc726ce..99f4ed8 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfoJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/RefCountChangeInfoJUnitTest.java
@@ -43,65 +43,92 @@ public class RefCountChangeInfoJUnitTest {
     String notOwner1 = new String("notInfo1");
 
     RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
+    RefCountChangeInfo refInfo2 = new RefCountChangeInfo(true, 1, notOwner1);
 
     assertEquals(owner1, refInfo1.getOwner());
+    assertEquals(notOwner1, refInfo2.getOwner());
+    assertFalse(refInfo1.getOwner().equals(refInfo2.getOwner()));
 
-    try {
-      assertEquals(owner1, notOwner1);
-      fail("Expected owner1 != notOwner1");
-    } catch (AssertionError e) {
-      // Ignore expected error
-    }
+  }
+
+  @Test
+  public void testNullOwner() {
+
+    String owner1 = null;
+    String notOwner1 = new String("notInfo1");
+
+    RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
+    RefCountChangeInfo refInfo2 = new RefCountChangeInfo(true, 1, notOwner1);
+    assertFalse(isOwnerNull(refInfo2.getOwner()));
+    assertTrue(hasStringLit(refInfo2.toString(), " owner="));
+
+    assertEquals(owner1, refInfo1.getOwner());
+    assertEquals(notOwner1, refInfo2.getOwner());
+    assertTrue(isOwnerNull(refInfo1.getOwner()));
+    assertFalse(hasStringLit(refInfo1.toString(), " owner="));
 
   }
 
+  private boolean isOwnerNull(Object own1) {
+    return own1 == null;
+  }
+  
+  private boolean hasStringLit(String str, String has) {
+    if(str.indexOf(has) == -1) return false;
+    return true;
+  }
+  
   @Test
-  public void testGetDupCount() {
+  public void testGetUseCount() {
 
     String owner1 = new String("Info1");
     String owner2 = new String("Info2");
 
     RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
-    assertEquals(0, refInfo1.getDupCount());
+    assertEquals(0, refInfo1.getUseCount());
 
     RefCountChangeInfo refInfo2 = new RefCountChangeInfo(true, 1, owner1);
-    assertTrue(refInfo1.isDuplicate(refInfo2));
-    assertEquals(1, refInfo1.getDupCount());
+    assertTrue(refInfo1.isSameCaller(refInfo2));
+    refInfo1.incUseCount();
+    assertEquals(1, refInfo1.getUseCount());
 
     // owner not used in isDup
     RefCountChangeInfo refInfo3 = new RefCountChangeInfo(true, 1, owner2);
-    assertTrue(refInfo1.isDuplicate(refInfo3));
-    assertEquals(2, refInfo1.getDupCount());
+    assertTrue(refInfo1.isSameCaller(refInfo3));
+    refInfo1.incUseCount();
+    assertEquals(2, refInfo1.getUseCount());
 
     RefCountChangeInfo refInfo4 = new RefCountChangeInfo(false, 1, owner2);
-    assertFalse(refInfo1.isDuplicate(refInfo4));
-    assertEquals(2, refInfo1.getDupCount());
+    assertFalse(refInfo1.isSameCaller(refInfo4));
+    assertEquals(2, refInfo1.getUseCount());
 
   }
 
   @Test
-  public void testDecDupCount() {
+  public void testDecUseCount() {
 
     String owner1 = new String("Info1");
     String owner2 = new String("Info2");
 
     RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
-    assertEquals(0, refInfo1.getDupCount());
+    assertEquals(0, refInfo1.getUseCount());
 
     RefCountChangeInfo refInfo2 = new RefCountChangeInfo(true, 1, owner1);
-    assertTrue(refInfo1.isDuplicate(refInfo2));
-    assertEquals(1, refInfo1.getDupCount());
+    assertTrue(refInfo1.isSameCaller(refInfo2));
+    refInfo1.incUseCount();
+    assertEquals(1, refInfo1.getUseCount());
 
-    // owner not used in isDuplicate check
+    // owner not used in isSameCaller check
     RefCountChangeInfo refInfo3 = new RefCountChangeInfo(true, 1, owner2);
-    assertTrue(refInfo1.isDuplicate(refInfo3));
-    assertEquals(2, refInfo1.getDupCount());
+    assertTrue(refInfo1.isSameCaller(refInfo3));
+    refInfo1.incUseCount();
+    assertEquals(2, refInfo1.getUseCount());
 
-    refInfo1.decDupCount();
-    assertEquals(1, refInfo1.getDupCount());
+    refInfo1.decUseCount();
+    assertEquals(1, refInfo1.getUseCount());
 
-    refInfo1.decDupCount();
-    assertEquals(0, refInfo1.getDupCount());
+    refInfo1.decUseCount();
+    assertEquals(0, refInfo1.getUseCount());
 
   }
 
@@ -116,44 +143,65 @@ public class RefCountChangeInfoJUnitTest {
     assertEquals(refInfo1.toString(), refInfo2.toString());
 
     RefCountChangeInfo refInfo3 = new RefCountChangeInfo(false, 1, owner1);
-    try {
-      assertEquals(refInfo1.toString(), refInfo3.toString());
-      fail("expected refInfo1.toString() != refInfo3.toString()");
-    } catch (AssertionError e) {
-      // ignore expected IllegalArgumentException
-    }
+    assertFalse(refInfo1.toString().equals(refInfo3.toString()));
 
     RefCountChangeInfo refInfo4 = new RefCountChangeInfo(true, 2, owner1);
-    try {
-      assertEquals(refInfo1.toString(), refInfo4.toString());
-      fail("expected refInfo1.toString() != refInfo4.toString()");
-    } catch (AssertionError e) {
-      // ignore expected IllegalArgumentException
-    }
+    assertFalse(refInfo1.toString().equals(refInfo4.toString()));
 
   }
 
   @Test
-  public void testIsDuplicate() {
+  public void testisSameCaller() {
 
     String owner1 = new String("Info1");
     String owner2 = new String("Info2");
 
     RefCountChangeInfo refInfo1 = new RefCountChangeInfo(true, 1, owner1);
-    assertEquals(0, refInfo1.getDupCount());
+    assertEquals(0, refInfo1.getUseCount());
 
     RefCountChangeInfo refInfo2 = new RefCountChangeInfo(true, 1, owner1);
-    assertTrue(refInfo1.isDuplicate(refInfo2));
-    assertEquals(1, refInfo1.getDupCount());
+    assertTrue(refInfo1.isSameCaller(refInfo2));
+    refInfo1.incUseCount();
+    assertEquals(1, refInfo1.getUseCount());
+    String str = refInfo1.toString();
+    str = refInfo1.toString();
+    
+    assertTrue(hasStringLit(refInfo1.toString(), " useCount=1"));
 
-    RefCountChangeInfo refInfo3 = new RefCountChangeInfo(false, 1, owner1);
-    assertFalse(refInfo1.isDuplicate(refInfo3));
-    assertEquals(1, refInfo1.getDupCount());
 
+    RefCountChangeInfo refInfo3 = new RefCountChangeInfo(false, 1, owner1);
+    assertFalse(refInfo1.isSameCaller(refInfo3));
+    assertEquals(1, refInfo1.getUseCount());
+    
     RefCountChangeInfo refInfo4 = new RefCountChangeInfo(true, 1, owner2);
-    assertTrue(refInfo1.isDuplicate(refInfo4));
-    assertEquals(2, refInfo1.getDupCount());
+    assertTrue(refInfo1.isSameCaller(refInfo4));
+    refInfo1.incUseCount();
+    assertEquals(2, refInfo1.getUseCount());
+
+    assertTrue(hasStringLit(refInfo1.toString(), " useCount=2"));
+    
+    refInfo1.setStackTraceString("not_the_same");
+    assertFalse(refInfo1.isSameCaller(refInfo4));
+    assertEquals(2, refInfo1.getUseCount());
+    refInfo1.setStackTraceString(null);
+
+    refInfo1.setStackTraceString(new SameHashDifferentTrace());
+    refInfo4.setStackTraceString(new SameHashDifferentTrace());
+    assertFalse(refInfo1.isSameCaller(refInfo4));
+    assertEquals(2, refInfo1.getUseCount());
 
   }
 
+  class SameHashDifferentTrace {
+
+    public int hashCode() { 
+      return 1; 
+    }
+
+    public boolean equals(Object notused) { 
+      return false; 
+    }
+  }
+  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/412f2415/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorLifecycleListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorLifecycleListenerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorLifecycleListenerJUnitTest.java
deleted file mode 100755
index 2df8656..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorLifecycleListenerJUnitTest.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.offheap;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-/**
- * Tests SimpleMemoryAllocatorImpl.LifecycleListener
- * 
- * @author Kirk Lund
- */
-@Category(UnitTest.class)
-public class SimpleMemoryAllocatorLifecycleListenerJUnitTest {
-  
-  private final List<LifecycleListenerCallback> afterCreateCallbacks = new ArrayList<LifecycleListenerCallback>();
-  private final List<LifecycleListenerCallback> afterReuseCallbacks = new ArrayList<LifecycleListenerCallback>();
-  private final List<LifecycleListenerCallback> beforeCloseCallbacks = new ArrayList<LifecycleListenerCallback>();
-  private final TestLifecycleListener listener = new TestLifecycleListener(
-      this.afterCreateCallbacks, this.afterReuseCallbacks, this.beforeCloseCallbacks);
-  
-  @After
-  public void tearDown() throws Exception {
-    LifecycleListener.removeLifecycleListener(this.listener);
-    this.afterCreateCallbacks.clear();
-    this.afterReuseCallbacks.clear();
-    this.beforeCloseCallbacks.clear();
-    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
-  }
-
-  @Test
-  public void testAddRemoveListener() {
-    LifecycleListener.addLifecycleListener(this.listener);
-    LifecycleListener.removeLifecycleListener(this.listener);
-
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
-    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-
-    Assert.assertEquals(0, this.afterCreateCallbacks.size());
-    Assert.assertEquals(0, this.afterReuseCallbacks.size());
-    Assert.assertEquals(0, this.beforeCloseCallbacks.size());
-    
-    ma.close();
-
-    Assert.assertEquals(0, this.afterCreateCallbacks.size());
-    Assert.assertEquals(0, this.afterReuseCallbacks.size());
-    Assert.assertEquals(0, this.beforeCloseCallbacks.size());
-  }
-  
-  @Test
-  public void testCallbacksAreCalledAfterCreate() {
-    LifecycleListener.addLifecycleListener(this.listener);
-    
-    UnsafeMemoryChunk slab = new UnsafeMemoryChunk(1024); // 1k
-    SimpleMemoryAllocatorImpl ma = SimpleMemoryAllocatorImpl.create(new NullOutOfOffHeapMemoryListener(), new NullOffHeapMemoryStats(), new UnsafeMemoryChunk[]{slab});
-
-    Assert.assertEquals(1, this.afterCreateCallbacks.size());
-    Assert.assertEquals(0, this.afterReuseCallbacks.size());
-    Assert.assertEquals(0, this.beforeCloseCallbacks.size());
-    
-    ma.close();
-    
-    Assert.assertEquals(1, this.afterCreateCallbacks.size());
-    Assert.assertEquals(0, this.afterReuseCallbacks.size());
-    Assert.assertEquals(1, this.beforeCloseCallbacks.size());
-  }
-  
-  static final class LifecycleListenerCallback {
-    private final SimpleMemoryAllocatorImpl allocator;
-    private final long timeStamp;
-    private final Throwable creationTime;
-    LifecycleListenerCallback(SimpleMemoryAllocatorImpl allocator) {
-      this.allocator = allocator;
-      this.timeStamp = System.currentTimeMillis();
-      this.creationTime = new Exception();
-    }
-    SimpleMemoryAllocatorImpl getAllocator() {
-      return this.allocator;
-    }
-    Throwable getStackTrace() {
-      return this.creationTime;
-    }
-    long getCreationTime() {
-      return this.timeStamp;
-    }
-    @Override
-    public String toString() {
-      StringWriter sw = new StringWriter();
-      PrintWriter pw = new PrintWriter(sw);
-      pw.print(new StringBuilder().
-          append(super.toString()).
-          append(" created at ").
-          append(this.timeStamp).
-          append(" by ").toString());
-      this.creationTime.printStackTrace(pw);
-      return sw.toString();
-    }
-  }
-  
-  static class TestLifecycleListener implements LifecycleListener {
-    private final List<LifecycleListenerCallback> afterCreateCallbacks;
-    private final List<LifecycleListenerCallback> afterReuseCallbacks;
-    private final List<LifecycleListenerCallback> beforeCloseCallbacks;
-    TestLifecycleListener(List<LifecycleListenerCallback> afterCreateCallbacks,
-        List<LifecycleListenerCallback> afterReuseCallbacks,
-        List<LifecycleListenerCallback> beforeCloseCallbacks) {
-      this.afterCreateCallbacks = afterCreateCallbacks;
-      this.afterReuseCallbacks = afterReuseCallbacks;
-      this.beforeCloseCallbacks = beforeCloseCallbacks;
-    }
-    @Override
-    public void afterCreate(SimpleMemoryAllocatorImpl allocator) {
-      this.afterCreateCallbacks.add(new LifecycleListenerCallback(allocator));
-    }
-    @Override
-    public void afterReuse(SimpleMemoryAllocatorImpl allocator) {
-      this.afterReuseCallbacks.add(new LifecycleListenerCallback(allocator));
-    }
-    @Override
-    public void beforeClose(SimpleMemoryAllocatorImpl allocator) {
-      this.beforeCloseCallbacks.add(new LifecycleListenerCallback(allocator));
-    }
-  }
-}


[31/50] [abbrv] incubator-geode git commit: Merge branch 'feature/GEODE-654' into develop Merge PR #55

Posted by je...@apache.org.
Merge branch 'feature/GEODE-654' into develop
Merge PR #55


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

Branch: refs/heads/feature/GEODE-14
Commit: 49ffab7c00d0f261646d8e453fa65bffe0f66aa8
Parents: 9efb586 f86ddd4
Author: William Markito <wm...@pivotal.io>
Authored: Tue Dec 15 09:41:42 2015 -0800
Committer: William Markito <wm...@pivotal.io>
Committed: Tue Dec 15 09:42:02 2015 -0800

----------------------------------------------------------------------
 gemfire-site/website/content/index.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/49ffab7c/gemfire-site/website/content/index.html
----------------------------------------------------------------------


[13/50] [abbrv] incubator-geode git commit: Revert "Removing TCPConduit's Stub ID class"

Posted by je...@apache.org.
Revert "Removing TCPConduit's Stub ID class"

This reverts commit 5b35e43f93bfbf6d62eadf7979eb3a8b7f59b77e.

This commit was causing compilation failures.


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

Branch: refs/heads/feature/GEODE-14
Commit: 507f2f3a905e70fcabed9b83d4dc966ef3e9e6ec
Parents: f7670e1
Author: Dan Smith <up...@apache.org>
Authored: Fri Dec 11 16:49:38 2015 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Fri Dec 11 17:25:39 2015 -0800

----------------------------------------------------------------------
 .../internal/DistributionManager.java           |   8 +
 .../distributed/internal/StartupMessage.java    |   1 +
 .../internal/direct/DirectChannel.java          |  93 ++++++-
 .../internal/direct/MissingStubException.java   |  37 +++
 .../internal/direct/ShunnedMemberException.java |  34 ---
 .../internal/membership/MembershipManager.java  |  29 +-
 .../gms/mgr/GMSMembershipManager.java           | 197 +++++++++++--
 .../internal/i18n/ParentLocalizedStrings.java   |   6 +-
 .../gemfire/internal/tcp/Connection.java        | 117 ++++----
 .../gemfire/internal/tcp/ConnectionTable.java   |  91 +++---
 .../internal/tcp/MemberShunnedException.java    |   7 +-
 .../gemfire/internal/tcp/ServerDelegate.java    |   5 +-
 .../com/gemstone/gemfire/internal/tcp/Stub.java | 164 +++++++++++
 .../gemfire/internal/tcp/TCPConduit.java        | 274 ++++++++++++++++---
 .../internal/DistributionManagerDUnitTest.java  |   6 +-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |  31 ++-
 .../internal/tcp/ConnectionJUnitTest.java       |   3 +-
 17 files changed, 870 insertions(+), 233 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
index e3c342a..964845c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
@@ -91,6 +91,7 @@ import com.gemstone.gemfire.internal.sequencelog.MembershipLogger;
 import com.gemstone.gemfire.internal.tcp.Connection;
 import com.gemstone.gemfire.internal.tcp.ConnectionTable;
 import com.gemstone.gemfire.internal.tcp.ReenteredConnectException;
+import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantLock;
 
 /**
@@ -2714,6 +2715,13 @@ public class DistributionManager
       return false; // no peers, we are alone.
     }
 
+    // ensure we have stubs for everyone else
+    Iterator it = allOthers.iterator();
+    while (it.hasNext()) {
+      InternalDistributedMember member = (InternalDistributedMember)it.next();
+      membershipManager.getStubForMember(member);
+    }
+
     try {
       ok = op.sendStartupMessage(allOthers, STARTUP_TIMEOUT, equivs,
           redundancyZone, enforceUniqueZone());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
index 01f8c62..96f8b60 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
@@ -37,6 +37,7 @@ import com.gemstone.gemfire.internal.InternalDataSerializer.SerializerAttributes
 import com.gemstone.gemfire.internal.InternalInstantiator.InstantiatorAttributesHolder;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.tcp.Stub;
 
 /**
  * A message that is sent to all other distribution manager when

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
index d4df3bf..14ff923 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
@@ -38,7 +38,6 @@ import com.gemstone.gemfire.InternalGemFireException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.ToDataException;
 import com.gemstone.gemfire.cache.TimeoutException;
-import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
@@ -65,6 +64,7 @@ import com.gemstone.gemfire.internal.tcp.Connection;
 import com.gemstone.gemfire.internal.tcp.ConnectionException;
 import com.gemstone.gemfire.internal.tcp.MemberShunnedException;
 import com.gemstone.gemfire.internal.tcp.MsgStreamer;
+import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.tcp.TCPConduit;
 import com.gemstone.gemfire.internal.util.Breadcrumbs;
 import com.gemstone.gemfire.internal.util.concurrent.ReentrantSemaphore;
@@ -115,6 +115,13 @@ public class DirectChannel {
     }
     
     /**
+     * Returns the endpoint ID for the direct channel
+     */
+    public Stub getLocalStub() {
+      return conduit.getId();
+    }
+    
+    /**
      * when the initial number of members is known, this method is invoked
      * to ensure that connections to those members can be established in a
      * reasonable amount of time.  See bug 39848 
@@ -174,7 +181,7 @@ public class DirectChannel {
         this.groupOrderedSenderSem = new ReentrantSemaphore(MAX_GROUP_SENDERS);
         this.groupUnorderedSenderSem = new ReentrantSemaphore(MAX_GROUP_SENDERS);
         logger.info(LocalizedMessage.create(
-            LocalizedStrings.DirectChannel_GEMFIRE_P2P_LISTENER_STARTED_ON__0, conduit.getLocalAddr()));
+            LocalizedStrings.DirectChannel_GEMFIRE_P2P_LISTENER_STARTED_ON__0, conduit.getId()));
 
       }
       catch (ConnectionException ce) {
@@ -185,6 +192,48 @@ public class DirectChannel {
     }
 
  
+//   /**
+//    * 
+//    * @param addr destination for the message
+//    * @param stubMap map containing all the stubs
+//    * @param msg the original message
+//    * @param msgBuf the serialized message
+//    * @param directAck true if we need an ack
+//    * @param processorType the type (serialized, etc.)
+//    * @return if directAck, the Connection that needs the acknowledgment
+//    * @throws MissingStubException if we do not have a Stub for the recipient
+//    * @throws IOException if the message could not be sent
+//    */
+//   private Connection attemptSingleSend(MembershipManager mgr,
+//       InternalDistributedMember addr,
+//       DistributionMessage msg, ByteBuffer msgBuf,
+//       boolean directAck, int processorType)
+//       throws MissingStubException, IOException
+//   {
+//     if (!msg.deliverToSender() && localAddr.equals(addr))
+//       return null;
+
+//     if (addr == null)
+//       return null;
+//     Stub dest = mgr.getStubForMember(addr);
+//     if (dest == null) {
+//       // This should only happen if the member is no longer in the view.
+//       Assert.assertTrue(!mgr.memberExists(addr));
+//       throw new MissingStubException("No stub");
+//     }
+//     try {
+//       msgBuf.position(0); // fix for bug#30680
+//       Connection con = conduit.sendSync(dest, msgBuf, processorType, msg);
+//       if (directAck)
+//         return con;
+//       else
+//         return null;
+//     }
+//     catch(IOException t) {
+//       throw t;
+//       }
+//   }
+
   /**
    * Return how many concurrent operations should be allowed by default.
    * since 6.6, this has been raised to Integer.MAX value from the number
@@ -590,13 +639,22 @@ public class DirectChannel {
         continue;
       }
 
-      if (!mgr.memberExists(destination) || mgr.shutdownInProgress() || mgr.isShunned(destination)) {
+      Stub stub = mgr.getStubForMember(destination);
+      if (stub == null) {
         // This should only happen if the member is no longer in the view.
         if (logger.isTraceEnabled(LogMarker.DM)) {
-          logger.trace(LogMarker.DM, "Not a member: {}", destination);
+          logger.trace(LogMarker.DM, "No Stub for {}", destination);
         }
+        // The only time getStubForMember returns null is if we are
+        // shunning that member or we are shutting down.
+        // So the following assertion is wrong:
+        //Assert.assertTrue(!mgr.memberExists(destination));
+        // instead we should:
+        // Assert.assertTrue(mgr.shutdownInProgress() || mgr.isShunned(destination));
+        //but this is not worth doing and isShunned is not public.
+        // SO the assert has been deadcoded.
         if (ce == null) ce = new ConnectExceptions();
-        ce.addFailure(destination, new ShunnedMemberException(LocalizedStrings.DirectChannel_SHUNNING_0.toLocalizedString()));
+        ce.addFailure(destination, new MissingStubException(LocalizedStrings.DirectChannel_NO_STUB_0.toLocalizedString()));
       }
       else {
         try {
@@ -604,8 +662,8 @@ public class DirectChannel {
           if (ackTimeout > 0) {
             startTime = System.currentTimeMillis();
           }
-          Connection con = conduit.getConnection(destination, preserveOrder,
-              retry, startTime, ackTimeout, ackSDTimeout);
+          Connection con = conduit.getConnection(destination, stub,
+              preserveOrder, retry, startTime, ackTimeout, ackSDTimeout);
           
           con.setInUse(true, startTime, 0, 0, null); // fix for bug#37657
           cons.add(con);
@@ -765,7 +823,7 @@ public class DirectChannel {
   }
 
   
-  public void receive(DistributionMessage msg, int bytesRead) {
+  public void receive(DistributionMessage msg, int bytesRead, Stub connId) {
     if (disconnected) {
       return;
     }
@@ -786,6 +844,10 @@ public class DirectChannel {
     }
   }
 
+//  public void newMemberConnected(InternalDistributedMember member, Stub id) {
+//    receiver.newMemberConnected(member, id);
+//  }
+
   public InternalDistributedMember getLocalAddress() {
     return this.localAddr;
   }
@@ -868,6 +930,13 @@ public class DirectChannel {
     }
   }
   
+  /** Create a TCPConduit stub from a JGroups InternalDistributedMember */
+  public Stub createConduitStub(InternalDistributedMember addr) {
+    int port = addr.getDirectChannelPort();
+    Stub stub = new Stub(addr.getInetAddress(), port, addr.getVmViewId());
+    return stub;
+  }
+  
   public void closeEndpoint(InternalDistributedMember member, String reason) {
     closeEndpoint(member, reason, true);
   }
@@ -879,7 +948,7 @@ public class DirectChannel {
   public void closeEndpoint(InternalDistributedMember member, String reason, boolean notifyDisconnect) {
     TCPConduit tc = this.conduit;
     if (tc != null) {
-      tc.removeEndpoint(member, reason, notifyDisconnect);
+      tc.removeEndpoint(createConduitStub(member), reason, notifyDisconnect);
     }
   }
 
@@ -893,7 +962,7 @@ public class DirectChannel {
    *    the map to add the state to
    * @since 5.1
    */
-  public void getChannelStates(DistributedMember member, Map result)
+  public void getChannelStates(Stub member, Map result)
   {
     TCPConduit tc = this.conduit;
     if (tc != null) {
@@ -905,7 +974,7 @@ public class DirectChannel {
    * wait for the given connections to process the number of messages
    * associated with the connection in the given map
    */
-  public void waitForChannelState(DistributedMember member, Map channelState)
+  public void waitForChannelState(Stub member, Map channelState)
     throws InterruptedException
   {
     if (Thread.interrupted()) throw new InterruptedException();
@@ -918,7 +987,7 @@ public class DirectChannel {
   /**
    * returns true if there are still receiver threads for the given member
    */
-  public boolean hasReceiversFor(DistributedMember mbr) {
+  public boolean hasReceiversFor(Stub mbr) {
     return this.conduit.hasReceiversFor(mbr);
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java
new file mode 100644
index 0000000..49b4486
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed.internal.direct;
+
+import com.gemstone.gemfire.GemFireCheckedException;
+
+/**
+ * Exception thrown when the TCPConduit is unable to acquire a stub
+ * for the given recipient.
+ * 
+ * @author jpenney
+ *
+ */
+public class MissingStubException extends GemFireCheckedException
+{
+
+  private static final long serialVersionUID = -6455664684151074915L;
+
+  public MissingStubException(String msg) {
+    super(msg);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java
deleted file mode 100644
index 59db762..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.distributed.internal.direct;
-
-import com.gemstone.gemfire.GemFireCheckedException;
-
-/**
- * Exception thrown when a member is no longer in the distributed system
- * 
- */
-public class ShunnedMemberException extends GemFireCheckedException
-{
-
-  private static final long serialVersionUID = -6455664684151074915L;
-
-  public ShunnedMemberException(String msg) {
-    super(msg);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
index 7416efa..a46680b 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
@@ -27,6 +27,7 @@ import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
+import com.gemstone.gemfire.internal.tcp.Stub;
 
 /**
  * A MembershipManager is responsible for reporting a MemberView, as well as
@@ -73,7 +74,7 @@ public interface MembershipManager {
    * @param m the member
    * @return true if it still exists
    */
-  public boolean memberExists(DistributedMember m);
+  public boolean memberExists(InternalDistributedMember m);
   
   /**
    * Is this manager still connected?  If it has not been initialized, this
@@ -142,6 +143,25 @@ public interface MembershipManager {
   throws NotSerializableException;
   
   /**
+   * Return a {@link Stub} referring to the given member.  A <em>null</em> may
+   * be returned if the system is not employing stubs for communication.
+   * 
+   * @param m the member
+   * @return the stub
+   */
+  public Stub getStubForMember(InternalDistributedMember m);
+  
+  /**
+   * Return a {@link InternalDistributedMember} associated with the given Stub.  This
+   * method may return a null if Stubs are not being used.
+   * @param s Stub to look up
+   * @param validated true if member must be in the current view
+   * @return the member associated with the given stub, if any
+   */
+  public InternalDistributedMember getMemberForStub(Stub s, boolean validated);
+  
+  
+  /**
    * Indicates to the membership manager that the system is shutting down.
    * Typically speaking, this means that new connection attempts are to be
    * ignored and disconnect failures are to be (more) tolerated.
@@ -266,7 +286,7 @@ public interface MembershipManager {
    */
   public void warnShun(DistributedMember mbr);
   
-  public boolean addSurpriseMember(DistributedMember mbr);
+  public boolean addSurpriseMember(DistributedMember mbr, Stub stub);
   
   /** if a StartupMessage is going to reject a new member, this should be used
    * to make sure we don't keep that member on as a "surprise member"
@@ -287,11 +307,6 @@ public interface MembershipManager {
    * @return true if the member is a surprise member
    */
   public boolean isSurpriseMember(DistributedMember m);
-  
-  /**
-   * Returns true if the member is being shunned
-   */
-  public boolean isShunned(DistributedMember m);
 
   /**
    * Forces use of UDP for communications in the current thread.  UDP is

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index 7be0a3a..0b7a544 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -94,6 +94,7 @@ import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.shared.StringPrintWriter;
 import com.gemstone.gemfire.internal.tcp.ConnectExceptions;
 import com.gemstone.gemfire.internal.tcp.MemberShunnedException;
+import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.util.Breadcrumbs;
 
 public class GMSMembershipManager implements MembershipManager, Manager
@@ -155,6 +156,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     boolean crashed;
     String reason;
     DistributionMessage dmsg;
+    Stub stub;
     NetView gmsView;
     
     @Override
@@ -163,7 +165,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       sb.append("kind=");
       switch (kind) {
       case SURPRISE_CONNECT:
-        sb.append("connect; member = <" + member + ">");
+        sb.append("connect; member = <" + member + ">; stub = " + stub);
         break;
       case VIEW:
         String text = gmsView.toString();
@@ -182,10 +184,12 @@ public class GMSMembershipManager implements MembershipManager, Manager
     /**
      * Create a surprise connect event
      * @param member the member connecting
+     * @param id the stub
      */
-    StartupEvent(final InternalDistributedMember member) {
+    StartupEvent(final InternalDistributedMember member, final Stub id) {
       this.kind = SURPRISE_CONNECT;
       this.member = member;
+      this.stub = id;
     }
     /**
      * Indicate if this is a surprise connect event
@@ -278,6 +282,24 @@ public class GMSMembershipManager implements MembershipManager, Manager
   volatile boolean hasJoined;
   
   /**
+   * a map keyed on InternalDistributedMember, values are Stubs that represent direct
+   * channels to other systems
+   * 
+   * Accesses must be under the read or write lock of {@link #latestViewLock}.
+   */
+  protected final Map<InternalDistributedMember, Stub> memberToStubMap = 
+      new ConcurrentHashMap<InternalDistributedMember, Stub>();
+
+  /**
+   * a map of direct channels (Stub) to InternalDistributedMember. key instanceof Stub
+   * value instanceof InternalDistributedMember
+   * 
+   * Accesses must be under the read or write lock of {@link #latestViewLock}.
+   */
+  protected final Map<Stub, InternalDistributedMember> stubToMemberMap = 
+      new ConcurrentHashMap<Stub, InternalDistributedMember>();
+  
+  /**
    * Members of the distributed system that we believe have shut down.
    * Keys are instances of {@link InternalDistributedMember}, values are 
    * Longs indicating the time this member was shunned.
@@ -525,6 +547,12 @@ public class GMSMembershipManager implements MembershipManager, Manager
           }
         }
 
+        // fix for bug #42006, lingering old identity
+        Object oldStub = this.memberToStubMap.remove(m);
+        if (oldStub != null) {
+          this.stubToMemberMap.remove(oldStub);
+        }
+
         if (shutdownInProgress()) {
           addShunnedMember(m);
           continue; // no additions processed after shutdown begins
@@ -778,6 +806,9 @@ public class GMSMembershipManager implements MembershipManager, Manager
     
     if (directChannel != null) {
       directChannel.setLocalAddr(address);
+      Stub stub = directChannel.getLocalStub();
+      memberToStubMap.put(address, stub);
+      stubToMemberMap.put(stub, address);
     }
 
     this.hasJoined = true;
@@ -874,15 +905,17 @@ public class GMSMembershipManager implements MembershipManager, Manager
   /**
    * Process a surprise connect event, or place it on the startup queue.
    * @param member the member
+   * @param stub its stub
    */
   protected void handleOrDeferSurpriseConnect(InternalDistributedMember member) {
+    Stub stub = new Stub(member.getInetAddress(), member.getDirectChannelPort(), member.getVmViewId());
     synchronized (startupLock) {
       if (!processingEvents) {
-        startupMessages.add(new StartupEvent(member));
+        startupMessages.add(new StartupEvent(member, stub));
         return;
       }
     }
-    processSurpriseConnect(member);
+    processSurpriseConnect(member, stub);
   }
   
   public void startupMessageFailed(DistributedMember mbr, String failureMessage) {
@@ -908,9 +941,12 @@ public class GMSMembershipManager implements MembershipManager, Manager
    * been added, simply returns; else adds the member.
    * 
    * @param dm the member joining
+   * @param stub the member's stub
    */
-  public boolean addSurpriseMember(DistributedMember dm) {
+  public boolean addSurpriseMember(DistributedMember dm, 
+      Stub stub) {
     final InternalDistributedMember member = (InternalDistributedMember)dm;
+    Stub s = null;
     boolean warn = false;
     
     latestViewLock.writeLock().lock();
@@ -973,6 +1009,16 @@ public class GMSMembershipManager implements MembershipManager, Manager
           startCleanupTimer();
         } // cleanupTimer == null
 
+        // fix for bug #42006, lingering old identity
+        Object oldStub = this.memberToStubMap.remove(member);
+        if (oldStub != null) {
+          this.stubToMemberMap.remove(oldStub);
+        }
+
+        s = stub == null ? getStubForMember(member) : stub;
+        // Make sure that channel information is consistent
+        addChannel(member, s);
+
         // Ensure that the member is accounted for in the view
         // Conjure up a new view including the new member. This is necessary
         // because we are about to tell the listener about a new member, so
@@ -1108,7 +1154,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
 
         // If it's a new sender, wait our turn, generate the event
         if (isNew) {
-          shunned = !addSurpriseMember(m);
+          shunned = !addSurpriseMember(m, getStubForMember(m));
         } // isNew
       }
 
@@ -1120,7 +1166,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     if (shunned) { // bug #41538 - shun notification must be outside synchronization to avoid hanging
       warnShun(m);
       logger.info("Membership: Ignoring message from shunned member <{}>:{}", m, msg);
-      throw new MemberShunnedException(m);
+      throw new MemberShunnedException(getStubForMember(m));
     }
     
     listener.messageReceived(msg);
@@ -1202,11 +1248,13 @@ public class GMSMembershipManager implements MembershipManager, Manager
    * grabbed a stable view if this is really a new member.
    * 
    * @param member
+   * @param stub
    */
   private void processSurpriseConnect(
-      InternalDistributedMember member) 
+      InternalDistributedMember member, 
+      Stub stub) 
   {
-    addSurpriseMember(member);
+    addSurpriseMember(member, stub);
   }
   
   /**
@@ -1228,7 +1276,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       processView(o.gmsView.getViewId(), o.gmsView);
     }
     else if (o.isSurpriseConnect()) { // connect
-      processSurpriseConnect(o.member);
+      processSurpriseConnect(o.member, o.stub);
     }
     
     else // sanity
@@ -1402,7 +1450,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     }
   }
 
-  public boolean memberExists(DistributedMember m) {
+  public boolean memberExists(InternalDistributedMember m) {
     latestViewLock.readLock().lock();
     NetView v = latestView;
     latestViewLock.readLock().unlock();
@@ -1477,6 +1525,12 @@ public class GMSMembershipManager implements MembershipManager, Manager
       directChannel.emergencyClose();
     }
     
+    // could we guarantee not to allocate objects?  We're using Darrel's 
+    // factory, so it's possible that an unsafe implementation could be
+    // introduced here.
+//    stubToMemberMap.clear();
+//    memberToStubMap.clear();
+    
     if (DEBUG) {
       System.err.println("DEBUG: done closing GroupMembershipService");
     }
@@ -1713,7 +1767,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       allDestinations = true;
       latestViewLock.writeLock().lock();
       try {
-        List<InternalDistributedMember> keySet = latestView.getMembers();
+        Set keySet = memberToStubMap.keySet();
         keys = new InternalDistributedMember[keySet.size()];
         keys = (InternalDistributedMember[])keySet.toArray(keys);
       } finally {
@@ -1966,6 +2020,80 @@ public class GMSMembershipManager implements MembershipManager, Manager
     // not currently supported by this manager
   }
   
+  /**
+   * Get or create stub for given member
+   */
+  public Stub getStubForMember(InternalDistributedMember m)
+  {
+    if (shutdownInProgress) {
+      throw new DistributedSystemDisconnectedException(LocalizedStrings.GroupMembershipService_DISTRIBUTEDSYSTEM_IS_SHUTTING_DOWN.toLocalizedString(), services.getShutdownCause());
+    }
+
+    if (services.getConfig().getDistributionConfig().getDisableTcp()) {
+      return new Stub(m.getInetAddress(), m.getPort(), m.getVmViewId());
+    }
+    
+    // Return existing one if it is already in place
+    Stub result;
+    result = (Stub)memberToStubMap.get(m);
+    if (result != null)
+      return result;
+
+    latestViewLock.writeLock().lock();
+    try {
+      // Do all of this work in a critical region to prevent
+      // members from slipping in during shutdown
+      if (shutdownInProgress())
+        return null; // don't try to create a stub during shutdown
+      if (isShunned(m))
+        return null; // don't let zombies come back to life
+      
+      // OK, create one.  Update the table to reflect the creation.
+      result = directChannel.createConduitStub(m);
+      addChannel(m, result);
+    } finally {
+      latestViewLock.writeLock().unlock();
+    }
+   return result;
+  }
+
+  public InternalDistributedMember getMemberForStub(Stub s, boolean validated)
+  {
+    latestViewLock.writeLock().lock();
+    try {
+      if (shutdownInProgress) {
+        throw new DistributedSystemDisconnectedException(LocalizedStrings.GroupMembershipService_DISTRIBUTEDSYSTEM_IS_SHUTTING_DOWN.toLocalizedString(), services.getShutdownCause());
+      }
+      InternalDistributedMember result = (InternalDistributedMember)
+          stubToMemberMap.get(s);
+      if (result != null) {
+        if (validated && !this.latestView.contains(result)) {
+          // Do not return this member unless it is in the current view.
+          if (!surpriseMembers.containsKey(result)) {
+            // if not a surprise member, this stub is lingering and should be removed
+            stubToMemberMap.remove(s);
+            memberToStubMap.remove(result);
+          }
+          result = null;
+          // fall through to see if there is a newer member using the same direct port
+        }
+      }
+      if (result == null) {
+        // it may have not been added to the stub->idm map yet, so check the current view
+        for (InternalDistributedMember idm: latestView.getMembers()) {
+          if (GMSUtil.compareAddresses(idm.getInetAddress(), s.getInetAddress()) == 0
+              && idm.getDirectChannelPort() == s.getPort()) {
+            addChannel(idm, s);
+            return idm;
+          }
+        }
+      }
+      return result;
+    } finally {
+      latestViewLock.writeLock().unlock();
+    }
+  }
+
   public void setShutdown()
   {
     latestViewLock.writeLock().lock();
@@ -1981,6 +2109,24 @@ public class GMSMembershipManager implements MembershipManager, Manager
     return shutdownInProgress || (dm != null && dm.shutdownInProgress());
   }
   
+  /**
+   * Add a mapping from the given member to the given stub. Must
+   * be called with {@link #latestViewLock} held.
+   * 
+   * @param member
+   * @param theChannel
+   */
+  protected void addChannel(InternalDistributedMember member, Stub theChannel)
+  {
+    if (theChannel != null) {
+      // Don't overwrite existing stub information with a null
+      this.memberToStubMap.put(member, theChannel);
+
+      // Can't create reverse mapping if the stub is null
+      this.stubToMemberMap.put(theChannel, member);
+    }
+  }
+
 
   /**
    * Clean up and create consistent new view with member removed.
@@ -1991,6 +2137,12 @@ public class GMSMembershipManager implements MembershipManager, Manager
   protected void destroyMember(final InternalDistributedMember member,
       boolean crashed, final String reason) {
     
+    // Clean up the maps
+    Stub theChannel = (Stub)memberToStubMap.remove(member);
+    if (theChannel != null) {
+      this.stubToMemberMap.remove(theChannel);
+    }
+    
     // Make sure it is removed from the view
     latestViewLock.writeLock().lock();
     try {
@@ -2213,11 +2365,12 @@ public class GMSMembershipManager implements MembershipManager, Manager
   /* non-thread-owned serial channels and high priority channels are not
    * included
    */
-  public Map getChannelStates(DistributedMember member, boolean includeMulticast) {
+  public Map getMessageState(DistributedMember member, boolean includeMulticast) {
     Map result = new HashMap();
+    Stub stub = (Stub)memberToStubMap.get(member);
     DirectChannel dc = directChannel;
-    if (dc != null) {
-      dc.getChannelStates(member, result);
+    if (stub != null && dc != null) {
+      dc.getChannelStates(stub, result);
     }
     services.getMessenger().getMessageState((InternalDistributedMember)member, result, includeMulticast);
     return result;
@@ -2228,8 +2381,15 @@ public class GMSMembershipManager implements MembershipManager, Manager
   {
     if (Thread.interrupted()) throw new InterruptedException();
     DirectChannel dc = directChannel;
-    if (dc != null) {
-      dc.waitForChannelState(otherMember, channelState);
+    Stub stub;
+    latestViewLock.writeLock().lock();
+    try {
+      stub = (Stub)memberToStubMap.get(otherMember);
+    } finally {
+      latestViewLock.writeLock().unlock();
+    }
+    if (dc != null && stub != null) {
+      dc.waitForChannelState(stub, state);
     }
     services.getMessenger().waitForMessageState((InternalDistributedMember)otherMember, state);
   }
@@ -2245,6 +2405,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     boolean result = false;
     DirectChannel dc = directChannel;
     InternalDistributedMember idm = (InternalDistributedMember)mbr;
+    Stub stub = new Stub(idm.getInetAddress(), idm.getPort(), idm.getVmViewId());
     int memberTimeout = this.services.getConfig().getDistributionConfig().getMemberTimeout();
     long pauseTime = (memberTimeout < 1000) ? 100 : memberTimeout / 10;
     boolean wait;
@@ -2252,7 +2413,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     do {
       wait = false;
       if (dc != null) {
-        if (dc.hasReceiversFor(idm)) {
+        if (dc.hasReceiversFor(stub)) {
           wait = true;
         }
         if (wait && logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/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 780fe18..7bb97b9 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
@@ -1109,7 +1109,7 @@ class ParentLocalizedStrings {
   public static final StringId TCPConduit_ENDING_RECONNECT_ATTEMPT_BECAUSE_0_HAS_DISAPPEARED = new StringId(2086, "Ending reconnect attempt because {0} has disappeared.");
   public static final StringId TCPConduit_ENDING_RECONNECT_ATTEMPT_TO_0_BECAUSE_SHUTDOWN_HAS_STARTED = new StringId(2087, "Ending reconnect attempt to {0} because shutdown has started.");
   public static final StringId TCPConduit_ERROR_SENDING_MESSAGE_TO_0_WILL_REATTEMPT_1 = new StringId(2088, "Error sending message to {0} (will reattempt): {1}");
-  public static final StringId TCPConduit_EXCEPTION_CREATING_SERVERSOCKET = new StringId(2089, "While creating ServerSocket on port {0} with address {1}");
+  public static final StringId TCPConduit_EXCEPTION_CREATING_SERVERSOCKET = new StringId(2089, "While creating ServerSocket and Stub on port {0} with address {1}");
   public static final StringId TCPConduit_EXCEPTION_PARSING_P2PIDLECONNECTIONTIMEOUT = new StringId(2090, "exception parsing p2p.idleConnectionTimeout");
   public static final StringId TCPConduit_EXCEPTION_PARSING_P2PTCPBUFFERSIZE = new StringId(2091, "exception parsing p2p.tcpBufferSize");
   public static final StringId TCPConduit_FAILED_TO_ACCEPT_CONNECTION_FROM_0_BECAUSE_1 = new StringId(2092, "Failed to accept connection from {0} because {1}");
@@ -1444,7 +1444,7 @@ class ParentLocalizedStrings {
   public static final StringId Connection_DETECTED_WRONG_VERSION_OF_GEMFIRE_PRODUCT_DURING_HANDSHAKE_EXPECTED_0_BUT_FOUND_1 = new StringId(2432, "Detected wrong version of GemFire product during handshake. Expected  {0}  but found  {1}");
   public static final StringId Connection_FORCED_DISCONNECT_SENT_TO_0 = new StringId(2433, "Forced disconnect sent to  {0}");
   public static final StringId Connection_HANDSHAKE_FAILED = new StringId(2434, "Handshake failed");
-  public static final StringId Connection_MEMBER_LEFT_THE_GROUP = new StringId(2435, "Member {0}  left the group");
+  public static final StringId Connection_MEMBER_FOR_STUB_0_LEFT_THE_GROUP = new StringId(2435, "Member for stub  {0}  left the group");
   public static final StringId Connection_NOT_CONNECTED_TO_0 = new StringId(2436, "Not connected to  {0}");
   public static final StringId Connection_NULL_CONNECTIONTABLE = new StringId(2437, "Null ConnectionTable");
   public static final StringId Connection_SOCKET_HAS_BEEN_CLOSED = new StringId(2438, "socket has been closed");
@@ -1542,7 +1542,7 @@ class ParentLocalizedStrings {
   public static final StringId DefaultQuery_WHEN_QUERYING_A_PARTITIONED_REGION_THE_PROJECTIONS_MUST_NOT_REFERENCE_ANY_REGIONS = new StringId(2530, "When querying a Partitioned Region, the projections must not reference any regions");
   public static final StringId DestroyMessage_FAILED_SENDING_0 = new StringId(2531, "Failed sending < {0} >");
   public static final StringId DirectChannel_COMMUNICATIONS_DISCONNECTED = new StringId(2532, "communications disconnected");
-  public static final StringId DirectChannel_SHUNNING_0 = new StringId(2533, "Member is being shunned: {0}");
+  public static final StringId DirectChannel_NO_STUB_0 = new StringId(2533, "No stub {0}");
   public static final StringId DirectChannel_UNKNOWN_ERROR_SERIALIZING_MESSAGE = new StringId(2534, "Unknown error serializing message");
   public static final StringId DiskEntry_AN_IOEXCEPTION_WAS_THROWN_WHILE_SERIALIZING = new StringId(2535, "An IOException was thrown while serializing.");
   public static final StringId DiskEntry_DISK_REGION_IS_NULL = new StringId(2536, "Disk region is null");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
index 74660da..f918812 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
@@ -50,7 +50,6 @@ import org.apache.logging.log4j.Logger;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.CacheClosedException;
-import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.ConflationKey;
 import com.gemstone.gemfire.distributed.internal.DM;
@@ -73,6 +72,7 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.ByteArrayDataInput;
 import com.gemstone.gemfire.internal.DSFIDFactory;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
+import com.gemstone.gemfire.internal.SocketCloser;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.SystemTimer;
@@ -222,6 +222,11 @@ public class Connection implements Runnable {
   /** the ID string of the conduit (for logging) */
   String conduitIdStr;
 
+  /** remoteId identifies the remote conduit's listener.  It does NOT
+     identify the "port" that this connection's socket is attached
+     to, which is a different thing altogether */
+  Stub remoteId;
+
   /** Identifies the java group member on the other side of the connection. */
   InternalDistributedMember remoteAddr;
 
@@ -796,7 +801,7 @@ public class Connection implements Runnable {
           }
           if (success) {
             if (this.isReceiver) {
-              needToClose = !owner.getConduit().getMembershipManager().addSurpriseMember(this.remoteAddr);
+              needToClose = !owner.getConduit().getMembershipManager().addSurpriseMember(this.remoteAddr, this.remoteId);
               if (needToClose) {
                 reason = "this member is shunned";
               }
@@ -840,7 +845,7 @@ public class Connection implements Runnable {
    * @param beingSick
    */
   private void asyncClose(boolean beingSick) {
-    // note: remoteAddr may be null if this is a receiver that hasn't finished its handshake
+    // note: remoteId may be null if this is a receiver that hasn't finished its handshake
     
     // we do the close in a background thread because the operation may hang if 
     // there is a problem with the network.  See bug #46659
@@ -1013,7 +1018,8 @@ public class Connection implements Runnable {
   protected static Connection createSender(final MembershipManager mgr,
                                            final ConnectionTable t,
                                            final boolean preserveOrder,
-                                           final DistributedMember remoteAddr,
+                                           final Stub key,
+                                           final InternalDistributedMember remoteAddr,
                                            final boolean sharedResource,
                                            final long startTime,
                                            final long ackTimeout,
@@ -1068,8 +1074,9 @@ public class Connection implements Runnable {
         }
         if (firstTime) {
           firstTime = false;
-          if (!mgr.memberExists(remoteAddr) || mgr.isShunned(remoteAddr) || mgr.shutdownInProgress()) {
-            throw new IOException("Member " + remoteAddr + " left the system");
+          InternalDistributedMember m = mgr.getMemberForStub(key, true);
+          if (m == null) {
+            throw new IOException("Member for stub " + key + " left the group");
           }
         }
         else {
@@ -1077,7 +1084,7 @@ public class Connection implements Runnable {
           // alert listener should not prevent cache operations from continuing
           if (AlertAppender.isThreadAlerting()) {
             // do not change the text of this exception - it is looked for in exception handlers
-            throw new IOException("Cannot form connection to alert listener " + remoteAddr);
+            throw new IOException("Cannot form connection to alert listener " + key);
           }
             
           // Wait briefly...
@@ -1090,19 +1097,20 @@ public class Connection implements Runnable {
             t.getConduit().getCancelCriterion().checkCancelInProgress(ie);
           }
           t.getConduit().getCancelCriterion().checkCancelInProgress(null);
-          if (giveUpOnMember(mgr, remoteAddr)) {
-            throw new IOException(LocalizedStrings.Connection_MEMBER_LEFT_THE_GROUP.toLocalizedString(remoteAddr));
+          InternalDistributedMember m = mgr.getMemberForStub(key, true);
+          if (m == null) {
+            throw new IOException(LocalizedStrings.Connection_MEMBER_FOR_STUB_0_LEFT_THE_GROUP.toLocalizedString(key));
           }
           if (!warningPrinted) {
             warningPrinted = true;
-            logger.warn(LocalizedMessage.create(LocalizedStrings.Connection_CONNECTION_ATTEMPTING_RECONNECT_TO_PEER__0, remoteAddr));
+            logger.warn(LocalizedMessage.create(LocalizedStrings.Connection_CONNECTION_ATTEMPTING_RECONNECT_TO_PEER__0, m));
           }          
           t.getConduit().stats.incReconnectAttempts();
         }
         //create connection
         try {
           conn = null;
-          conn = new Connection(mgr, t, preserveOrder, remoteAddr, sharedResource);
+          conn = new Connection(mgr, t, preserveOrder, key, remoteAddr, sharedResource);
         }
         catch (javax.net.ssl.SSLHandshakeException se) {
           // no need to retry if certificates were rejected
@@ -1110,7 +1118,8 @@ public class Connection implements Runnable {
         }
         catch (IOException ioe) {
           // Only give up if the member leaves the view.
-          if (giveUpOnMember(mgr, remoteAddr)) {
+          InternalDistributedMember m = mgr.getMemberForStub(key, true);
+          if (m == null) {
             throw ioe;
           }
           t.getConduit().getCancelCriterion().checkCancelInProgress(null);
@@ -1121,7 +1130,7 @@ public class Connection implements Runnable {
             connectionErrorLogged = true; // otherwise change to use 100ms intervals causes a lot of these
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.Connection_CONNECTION_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1,
-                new Object[] {sharedResource, preserveOrder, remoteAddr, ioe}));
+                new Object[] {sharedResource, preserveOrder, m, ioe}));
           }
         } // IOException
         finally {
@@ -1137,8 +1146,9 @@ public class Connection implements Runnable {
               // something went wrong while reading the handshake
               // and the socket was closed or this guy sent us a
               // ShutdownMessage
-              if (giveUpOnMember(mgr, remoteAddr)) {
-                throw new IOException(LocalizedStrings.Connection_MEMBER_LEFT_THE_GROUP.toLocalizedString(remoteAddr));
+              InternalDistributedMember m = mgr.getMemberForStub(key, true);
+              if (m == null) {
+                throw new IOException(LocalizedStrings.Connection_MEMBER_FOR_STUB_0_LEFT_THE_GROUP.toLocalizedString(key));
               }
               t.getConduit().getCancelCriterion().checkCancelInProgress(null);
               // no success but no need to log; just retry
@@ -1151,7 +1161,8 @@ public class Connection implements Runnable {
             throw e;
           }
           catch (ConnectionException e) {
-            if (giveUpOnMember(mgr, remoteAddr)) {
+            InternalDistributedMember m = mgr.getMemberForStub(key, true);
+            if (m == null) {
               IOException ioe = new IOException(LocalizedStrings.Connection_HANDSHAKE_FAILED.toLocalizedString());
               ioe.initCause(e);
               throw ioe;
@@ -1159,16 +1170,17 @@ public class Connection implements Runnable {
             t.getConduit().getCancelCriterion().checkCancelInProgress(null);
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.Connection_CONNECTION_HANDSHAKE_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1,
-                new Object[] {sharedResource, preserveOrder, remoteAddr ,e}));
+                new Object[] {sharedResource, preserveOrder, m,e}));
           }
           catch (IOException e) {
-            if (giveUpOnMember(mgr, remoteAddr)) {
+            InternalDistributedMember m = mgr.getMemberForStub(key, true);
+            if (m == null) {
               throw e;
             }
             t.getConduit().getCancelCriterion().checkCancelInProgress(null);
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.Connection_CONNECTION_HANDSHAKE_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1,
-                new Object[] {sharedResource, preserveOrder, remoteAddr ,e}));
+                new Object[] {sharedResource, preserveOrder, m,e}));
             if (!sharedResource && "Too many open files".equals(e.getMessage())) {
               t.fileDescriptorsExhausted();
             }
@@ -1208,7 +1220,7 @@ public class Connection implements Runnable {
     if (conn == null) {
       throw new ConnectionException(
         LocalizedStrings.Connection_CONNECTION_FAILED_CONSTRUCTION_FOR_PEER_0
-          .toLocalizedString(remoteAddr));
+          .toLocalizedString(mgr.getMemberForStub(key, true)));
     }
     if (preserveOrder && BATCH_SENDS) {
       conn.createBatchSendBuffer();
@@ -1216,15 +1228,12 @@ public class Connection implements Runnable {
     conn.finishedConnecting = true;
     return conn;
   }
-  
-  private static boolean giveUpOnMember(MembershipManager mgr, DistributedMember remoteAddr) {
-    return !mgr.memberExists(remoteAddr) || mgr.isShunned(remoteAddr) || mgr.shutdownInProgress();
-  }
 
-  private void setRemoteAddr(DistributedMember m) {
+  private void setRemoteAddr(InternalDistributedMember m, Stub stub) {
     this.remoteAddr = this.owner.getDM().getCanonicalId(m);
+    this.remoteId = stub;
     MembershipManager mgr = this.owner.owner.getMembershipManager();
-    mgr.addSurpriseMember(m);
+    mgr.addSurpriseMember(m, stub);
   }
   
   /** creates a new connection to a remote server.
@@ -1234,11 +1243,11 @@ public class Connection implements Runnable {
   private Connection(MembershipManager mgr,
                      ConnectionTable t,
                      boolean preserveOrder,
-                     DistributedMember remoteID,
+                     Stub key,
+                     InternalDistributedMember remoteAddr,
                      boolean sharedResource)
     throws IOException, DistributedSystemDisconnectedException
   {    
-    InternalDistributedMember remoteAddr = (InternalDistributedMember)remoteID;
     if (t == null) {
       throw new IllegalArgumentException(LocalizedStrings.Connection_CONNECTIONTABLE_IS_NULL.toLocalizedString());
     }
@@ -1246,7 +1255,7 @@ public class Connection implements Runnable {
     this.owner = t;
     this.sharedResource = sharedResource;
     this.preserveOrder = preserveOrder;
-    setRemoteAddr(remoteAddr);
+    setRemoteAddr(remoteAddr, key);
     this.conduitIdStr = this.owner.getConduit().getId().toString();
     this.handshakeRead = false;
     this.handshakeCancelled = false;
@@ -1256,7 +1265,7 @@ public class Connection implements Runnable {
 
     // connect to listening socket
 
-    InetSocketAddress addr = new InetSocketAddress(remoteAddr.getInetAddress(), remoteAddr.getDirectChannelPort());
+    InetSocketAddress addr = new InetSocketAddress(remoteId.getInetAddress(), remoteId.getPort());
     if (useNIO()) {
       SocketChannel channel = SocketChannel.open();
       this.owner.addConnectingSocket(channel.socket(), addr.getAddress());
@@ -1316,15 +1325,15 @@ public class Connection implements Runnable {
     else {
       if (TCPConduit.useSSL) {
         // socket = javax.net.ssl.SSLSocketFactory.getDefault()
-        //  .createSocket(remoteAddr.getInetAddress(), remoteAddr.getPort());
+        //  .createSocket(remoteId.getInetAddress(), remoteId.getPort());
         int socketBufferSize = sharedResource ? SMALL_BUFFER_SIZE : this.owner.getConduit().tcpBufferSize;
-        this.socket = SocketCreator.getDefaultInstance().connectForServer( remoteAddr.getInetAddress(), remoteAddr.getDirectChannelPort(), socketBufferSize );
+        this.socket = SocketCreator.getDefaultInstance().connectForServer( remoteId.getInetAddress(), remoteId.getPort(), socketBufferSize );
         // Set the receive buffer size local fields. It has already been set in the socket.
         setSocketBufferSize(this.socket, false, socketBufferSize, true);
         setSendBufferSize(this.socket);
       }
       else {
-        //socket = new Socket(remoteAddr.getInetAddress(), remoteAddr.getPort());
+        //socket = new Socket(remoteId.getInetAddress(), remoteId.getPort());
         Socket s = new Socket();
         this.socket = s;
         s.setTcpNoDelay(true);
@@ -1630,8 +1639,8 @@ public class Connection implements Runnable {
     // we can't wait for the reader thread when running in an IBM JRE.  See
     // bug 41889
     if (this.owner.owner.config.getEnableNetworkPartitionDetection() ||
-        this.owner.owner.getLocalAddr().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE ||
-        this.owner.owner.getLocalAddr().getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
+        this.owner.owner.getLocalId().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE ||
+        this.owner.owner.getLocalId().getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
       isIBM = "IBM Corporation".equals(System.getProperty("java.vm.vendor"));
     }
     {
@@ -1680,16 +1689,16 @@ public class Connection implements Runnable {
               // Only remove endpoint if sender.
               if (this.finishedConnecting) {
                 // only remove endpoint if our constructor finished
-                this.owner.removeEndpoint(this.remoteAddr, reason);
+                this.owner.removeEndpoint(this.remoteId, reason);
               }
             }
           }
           else {
-            this.owner.removeSharedConnection(reason, this.remoteAddr, this.preserveOrder, this);
+            this.owner.removeSharedConnection(reason, this.remoteId, this.preserveOrder, this);
           }
         }
         else if (!this.isReceiver) {
-          this.owner.removeThreadConnection(this.remoteAddr, this);
+          this.owner.removeThreadConnection(this.remoteId, this);
         }
       }
       else {
@@ -1697,10 +1706,10 @@ public class Connection implements Runnable {
         // has never added this Connection to its maps since
         // the calls in this block use our identity to do the removes.
         if (this.sharedResource) {
-          this.owner.removeSharedConnection(reason, this.remoteAddr, this.preserveOrder, this);
+          this.owner.removeSharedConnection(reason, this.remoteId, this.preserveOrder, this);
         }
         else if (!this.isReceiver) {
-          this.owner.removeThreadConnection(this.remoteAddr, this);
+          this.owner.removeThreadConnection(this.remoteId, this);
         }
       }
     }
@@ -1744,7 +1753,7 @@ public class Connection implements Runnable {
     } finally {
       // bug36060: do the socket close within a finally block
       if (logger.isDebugEnabled()) {
-        logger.debug("Stopping {} for {}", p2pReaderName(), remoteAddr);
+        logger.debug("Stopping {} for {}", p2pReaderName(), remoteId);
       }
       initiateSuspicionIfSharedUnordered();
       if (this.isReceiver) {
@@ -2329,7 +2338,8 @@ public class Connection implements Runnable {
                     .toLocalizedString(new Object[]{new Byte(HANDSHAKE_VERSION), new Byte(handShakeByte)}));
               }
               InternalDistributedMember remote = DSFIDFactory.readInternalDistributedMember(dis);
-              setRemoteAddr(remote);
+              Stub stub = new Stub(remote.getInetAddress()/*fix for bug 33615*/, remote.getDirectChannelPort(), remote.getVmViewId());
+              setRemoteAddr(remote, stub);
               Thread.currentThread().setName(LocalizedStrings.Connection_P2P_MESSAGE_READER_FOR_0.toLocalizedString(this.remoteAddr, this.socket.getPort()));
               this.sharedResource = dis.readBoolean();
               this.preserveOrder = dis.readBoolean();
@@ -2367,7 +2377,7 @@ public class Connection implements Runnable {
               }
               
               if (logger.isDebugEnabled()) {
-                logger.debug("{} remoteAddr is {} {}", p2pReaderName(), this.remoteAddr,
+                logger.debug("{} remoteId is {} {}", p2pReaderName(), this.remoteId,
                     (this.remoteVersion != null ? " (" + this.remoteVersion + ')' : ""));
               }
 
@@ -2545,7 +2555,7 @@ public class Connection implements Runnable {
     throws IOException, ConnectionException
   {
     if (!connected) {
-      throw new ConnectionException(LocalizedStrings.Connection_NOT_CONNECTED_TO_0.toLocalizedString(this.remoteAddr));
+      throw new ConnectionException(LocalizedStrings.Connection_NOT_CONNECTED_TO_0.toLocalizedString(this.remoteId));
     }
     if (this.batchFlusher != null) {
       batchSend(buffer);
@@ -2768,7 +2778,7 @@ public class Connection implements Runnable {
         if (this.disconnectRequested) {
           buffer.position(origBufferPos);
           // we have given up so just drop this message.
-          throw new ConnectionException(LocalizedStrings.Connection_FORCED_DISCONNECT_SENT_TO_0.toLocalizedString(this.remoteAddr));
+          throw new ConnectionException(LocalizedStrings.Connection_FORCED_DISCONNECT_SENT_TO_0.toLocalizedString(this.remoteId));
         }
         if (!force && !this.asyncQueuingInProgress) {
           // reset buffer since we will be sending it. This fixes bug 34832
@@ -2970,7 +2980,7 @@ public class Connection implements Runnable {
     }
     DM dm = this.owner.getDM();
     if (dm == null) {
-      this.owner.removeEndpoint(this.remoteAddr, LocalizedStrings.Connection_NO_DISTRIBUTION_MANAGER.toLocalizedString());
+      this.owner.removeEndpoint(this.remoteId, LocalizedStrings.Connection_NO_DISTRIBUTION_MANAGER.toLocalizedString());
       return;
     }
     dm.getMembershipManager().requestMemberRemoval(this.remoteAddr, 
@@ -2991,7 +3001,7 @@ public class Connection implements Runnable {
         return;
       }
     }
-    this.owner.removeEndpoint(this.remoteAddr, 
+    this.owner.removeEndpoint(this.remoteId, 
                               LocalizedStrings.Connection_FORCE_DISCONNECT_TIMED_OUT.toLocalizedString());
     if (dm.getOtherDistributionManagerIds().contains(this.remoteAddr)) {
       if (logger.isDebugEnabled()) {
@@ -3100,7 +3110,7 @@ public class Connection implements Runnable {
       stats.incAsyncThreads(-1);
       stats.incAsyncQueues(-1);
       if (logger.isDebugEnabled()) {
-        logger.debug("runNioPusher terminated id={} from {}/{}", conduitIdStr, remoteAddr, remoteAddr);
+        logger.debug("runNioPusher terminated id={} from {}/{}", conduitIdStr, remoteId, remoteAddr);
       }
     }
     } finally {
@@ -3827,7 +3837,8 @@ public class Connection implements Runnable {
                   throw new IllegalStateException(LocalizedStrings.Connection_DETECTED_WRONG_VERSION_OF_GEMFIRE_PRODUCT_DURING_HANDSHAKE_EXPECTED_0_BUT_FOUND_1.toLocalizedString(new Object[] {new Byte(HANDSHAKE_VERSION), new Byte(handShakeByte)}));
                 }
                 InternalDistributedMember remote = DSFIDFactory.readInternalDistributedMember(dis);
-                setRemoteAddr(remote);
+                Stub stub = new Stub(remote.getInetAddress()/*fix for bug 33615*/, remote.getDirectChannelPort(), remote.getVmViewId());
+                setRemoteAddr(remote, stub);
                 this.sharedResource = dis.readBoolean();
                 this.preserveOrder = dis.readBoolean();
                 this.uniqueId = dis.readLong();
@@ -3886,7 +3897,7 @@ public class Connection implements Runnable {
                 return;
               }
               if (logger.isDebugEnabled()) {
-                logger.debug("P2P handshake remoteAddr is {}{}", this.remoteAddr,
+                logger.debug("P2P handshake remoteId is {}{}", this.remoteId,
                     (this.remoteVersion != null ? " (" + this.remoteVersion + ')' : ""));
               }
               try {
@@ -4020,6 +4031,12 @@ public class Connection implements Runnable {
     this.accessed = true;
   }
 
+  /** returns the ConnectionKey stub representing the other side of
+      this connection (host:port) */
+  public final Stub getRemoteId() {
+    return remoteId;
+  }
+
   /** return the DM id of the guy on the other side of this connection.
    */
   public final InternalDistributedMember getRemoteAddress() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
index 3816efe..bac356c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
@@ -42,7 +42,6 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
@@ -61,7 +60,7 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 
 /** <p>ConnectionTable holds all of the Connection objects in a conduit.
     Connections represent a pipe between two endpoints represented
-    by generic DistributedMembers.</p>
+    by generic Stubs.</p>
 
     @author Bruce Schuchardt
     @author Darrel Schneider
@@ -346,7 +345,7 @@ public class ConnectionTable  {
   /**
    * Process a newly created PendingConnection
    * 
-   * @param id DistributedMember on which the connection is created
+   * @param id Stub on which the connection is created
    * @param sharedResource whether the connection is used by multiple threads
    * @param preserveOrder whether to preserve order
    * @param m map to add the connection to
@@ -358,7 +357,7 @@ public class ConnectionTable  {
    * @throws IOException if unable to connect
    * @throws DistributedSystemDisconnectedException
    */
-  private Connection handleNewPendingConnection(DistributedMember id, boolean sharedResource,
+  private Connection handleNewPendingConnection(Stub id, boolean sharedResource,
       boolean preserveOrder,
       Map m, PendingConnection pc, long startTime, long ackThreshold, long ackSAThreshold)
       throws IOException, DistributedSystemDisconnectedException
@@ -367,7 +366,7 @@ public class ConnectionTable  {
     Connection con = null;
     try {
       con = Connection.createSender(owner.getMembershipManager(), this, preserveOrder,
-                                    id,
+                                    id, this.owner.getMemberForStub(id, false),
                                     sharedResource,
                                     startTime, ackThreshold, ackSAThreshold);
       this.owner.stats.incSenders(sharedResource, preserveOrder);
@@ -443,7 +442,7 @@ public class ConnectionTable  {
    * unordered or conserve-sockets
    * note that unordered connections are currently always shared
    * 
-   * @param id the DistributedMember on which we are creating a connection
+   * @param id the Stub on which we are creating a connection
    * @param threadOwnsResources whether unordered conn is owned by the current thread
    * @param preserveOrder whether to preserve order
    * @param startTime the ms clock start time for the operation
@@ -453,7 +452,7 @@ public class ConnectionTable  {
    * @throws IOException if unable to create the connection
    * @throws DistributedSystemDisconnectedException
    */
-  private Connection getUnorderedOrConserveSockets(DistributedMember id, 
+  private Connection getUnorderedOrConserveSockets(Stub id, 
       boolean threadOwnsResources, boolean preserveOrder,
       long startTime, long ackTimeout, long ackSATimeout)
     throws IOException, DistributedSystemDisconnectedException
@@ -528,7 +527,7 @@ public class ConnectionTable  {
    * @throws IOException if the connection could not be created
    * @throws DistributedSystemDisconnectedException
    */
-  Connection getOrderedAndOwned(DistributedMember id, long startTime, long ackTimeout, long ackSATimeout) 
+  Connection getOrderedAndOwned(Stub id, long startTime, long ackTimeout, long ackSATimeout) 
       throws IOException, DistributedSystemDisconnectedException  {
     Connection result = null;
     
@@ -567,7 +566,7 @@ public class ConnectionTable  {
     // OK, we have to create a new connection.
     result = Connection.createSender(owner.getMembershipManager(), 
         this, true /* preserveOrder */, id,
-        false /* shared */,
+        this.owner.getMemberForStub(id, false), false /* shared */,
         startTime, ackTimeout, ackSATimeout);
     if (logger.isDebugEnabled()) {
       logger.debug("ConnectionTable: created an ordered connection: {}", result);
@@ -584,7 +583,7 @@ public class ConnectionTable  {
     
     ArrayList al = (ArrayList)this.threadConnectionMap.get(id);
     if (al == null) {
-      // First connection for this DistributedMember.  Make sure list for this
+      // First connection for this Stub.  Make sure list for this
       // stub is created if it isn't already there.
       al = new ArrayList();
       
@@ -652,7 +651,7 @@ public class ConnectionTable  {
   
   /**
    * Get a new connection
-   * @param id the DistributedMember on which to create the connection
+   * @param id the Stub on which to create the connection
    * @param preserveOrder whether order should be preserved
    * @param startTime the ms clock start time
    * @param ackTimeout the ms ack-wait-threshold, or zero
@@ -661,7 +660,7 @@ public class ConnectionTable  {
    * @throws java.io.IOException if the connection could not be created
    * @throws DistributedSystemDisconnectedException
    */
-  protected Connection get(DistributedMember id, boolean preserveOrder,
+  protected Connection get(Stub id, boolean preserveOrder,
       long startTime, long ackTimeout, long ackSATimeout) 
       throws java.io.IOException, DistributedSystemDisconnectedException
   {
@@ -839,38 +838,34 @@ public class ConnectionTable  {
   /**
    * Return true if our owner already knows that this endpoint is departing 
    */
-  protected boolean isEndpointShuttingDown(DistributedMember id) {
-    return giveUpOnMember(owner.getDM().getMembershipManager(), id);
+  protected boolean isEndpointShuttingDown(Stub stub) {
+    return this.owner.getMemberForStub(stub, true) == null;
   }
   
-  protected boolean giveUpOnMember(MembershipManager mgr, DistributedMember remoteAddr) {
-    return !mgr.memberExists(remoteAddr) || mgr.isShunned(remoteAddr) || mgr.shutdownInProgress();
-  }
-
   /** remove an endpoint and notify the membership manager of the departure */
-  protected void removeEndpoint(DistributedMember stub, String reason) {
+  protected void removeEndpoint(Stub stub, String reason) {
     removeEndpoint(stub, reason, true);
   }
 
-  protected void removeEndpoint(DistributedMember memberID, String reason, boolean notifyDisconnect) {
+  protected void removeEndpoint(Stub stub, String reason, boolean notifyDisconnect) {
     if (this.closed) {
       return;
     }
     boolean needsRemoval = false;
     synchronized (this.orderedConnectionMap) {
-      if (this.orderedConnectionMap.get(memberID) != null)
+      if (this.orderedConnectionMap.get(stub) != null)
         needsRemoval = true;
     }
     if (!needsRemoval) {
       synchronized (this.unorderedConnectionMap) {
-        if (this.unorderedConnectionMap.get(memberID) != null)
+        if (this.unorderedConnectionMap.get(stub) != null)
           needsRemoval = true;
       }
     }
     if (!needsRemoval) {
       ConcurrentMap cm = this.threadConnectionMap;
       if (cm != null) {
-        ArrayList al = (ArrayList)cm.get(memberID);
+        ArrayList al = (ArrayList)cm.get(stub);
         needsRemoval = al != null && al.size() > 0;
       }
     }
@@ -878,14 +873,14 @@ public class ConnectionTable  {
     if (needsRemoval) {
       InternalDistributedMember remoteAddress = null;
       synchronized (this.orderedConnectionMap) {
-        Object c = this.orderedConnectionMap.remove(memberID);
+        Object c = this.orderedConnectionMap.remove(stub);
         if (c instanceof Connection) {
           remoteAddress = ((Connection) c).getRemoteAddress();
         }
         closeCon(reason, c);
       }
       synchronized (this.unorderedConnectionMap) {
-        Object c = this.unorderedConnectionMap.remove(memberID);
+        Object c = this.unorderedConnectionMap.remove(stub);
         if (remoteAddress == null && (c instanceof Connection)) {
           remoteAddress = ((Connection) c).getRemoteAddress();
         }
@@ -895,7 +890,7 @@ public class ConnectionTable  {
       {
         ConcurrentMap cm = this.threadConnectionMap;
         if (cm != null) {
-          ArrayList al = (ArrayList)cm.remove(memberID);
+          ArrayList al = (ArrayList)cm.remove(stub);
           if (al != null) {
             synchronized (al) {
               for (Iterator it=al.iterator(); it.hasNext();) {
@@ -917,7 +912,7 @@ public class ConnectionTable  {
         for (Iterator it=connectingSockets.entrySet().iterator(); it.hasNext(); ) {
           Map.Entry entry = (Map.Entry)it.next();
           ConnectingSocketInfo info = (ConnectingSocketInfo)entry.getValue();
-          if (info.peerAddress.equals(((InternalDistributedMember)memberID).getInetAddress())) {
+          if (info.peerAddress.equals(stub.getInetAddress())) {
             toRemove.add(entry.getKey());
             it.remove();
           }
@@ -930,7 +925,7 @@ public class ConnectionTable  {
         }
         catch (IOException e) {
           if (logger.isDebugEnabled()) {
-            logger.debug("caught exception while trying to close connecting socket for {}", memberID, e);
+            logger.debug("caught exception while trying to close connecting socket for {}", stub, e);
           }
         }
       }
@@ -942,7 +937,7 @@ public class ConnectionTable  {
       synchronized (this.receivers) {
         for (Iterator it=receivers.iterator(); it.hasNext();) {
           Connection con = (Connection)it.next();
-          if (memberID.equals(con.getRemoteAddress())) {
+          if (stub.equals(con.getRemoteId())) {
             it.remove();
             toRemove.add(con);
           }
@@ -952,13 +947,10 @@ public class ConnectionTable  {
         Connection con = (Connection)it.next();
         closeCon(reason, con);
       }
+      // call memberDeparted after doing the closeCon calls
+      // so it can recursively call removeEndpoint
       if (notifyDisconnect) {
-        // Before the removal of TCPConduit Stub addresses this used
-        // to call MembershipManager.getMemberForStub, which checked
-        // for a shutdown in progress and threw this exception:
-        if (owner.getDM().shutdownInProgress()) {
-          throw new DistributedSystemDisconnectedException("Shutdown in progress", owner.getDM().getMembershipManager().getShutdownCause());
-        }
+        owner.getMemberForStub(stub, false);
       }
       
       if (remoteAddress != null) {
@@ -972,11 +964,11 @@ public class ConnectionTable  {
   }
   
   /** check to see if there are still any receiver threads for the given end-point */
-  protected boolean hasReceiversFor(DistributedMember endPoint) {
+  protected boolean hasReceiversFor(Stub endPoint) {
     synchronized (this.receivers) {
       for (Iterator it=receivers.iterator(); it.hasNext();) {
         Connection con = (Connection)it.next();
-        if (endPoint.equals(con.getRemoteAddress())) {
+        if (endPoint.equals(con.getRemoteId())) {
           return true;
         }
       }
@@ -984,7 +976,7 @@ public class ConnectionTable  {
     return false;
   }
   
-  private static void removeFromThreadConMap(ConcurrentMap cm, DistributedMember stub, Connection c) {
+  private static void removeFromThreadConMap(ConcurrentMap cm, Stub stub, Connection c) {
     if (cm != null) {
       ArrayList al = (ArrayList)cm.get(stub);
       if (al != null) {
@@ -994,7 +986,7 @@ public class ConnectionTable  {
       }
     }
   }
-  protected void removeThreadConnection(DistributedMember stub, Connection c) {
+  protected void removeThreadConnection(Stub stub, Connection c) {
     /*if (this.closed) {
       return;
     }*/
@@ -1009,7 +1001,7 @@ public class ConnectionTable  {
       } // synchronized
     } // m != null
   }
-  void removeSharedConnection(String reason, DistributedMember stub, boolean ordered, Connection c) {
+  void removeSharedConnection(String reason, Stub stub, boolean ordered, Connection c) {
     if (this.closed) {
       return;
     }
@@ -1062,7 +1054,7 @@ public class ConnectionTable  {
        Iterator it = m.entrySet().iterator();
        while (it.hasNext()) {
          Map.Entry me = (Map.Entry)it.next();
-         DistributedMember stub = (DistributedMember)me.getKey();
+         Stub stub = (Stub)me.getKey();
          Connection c = (Connection)me.getValue();
          removeFromThreadConMap(this.threadConnectionMap, stub, c);
          it.remove();
@@ -1087,7 +1079,7 @@ public class ConnectionTable  {
    * from being formed or new messages from being sent
    * @since 5.1
    */
-  protected void getThreadOwnedOrderedConnectionState(DistributedMember member,
+  protected void getThreadOwnedOrderedConnectionState(Stub member,
       Map result) {
 
     ConcurrentMap cm = this.threadConnectionMap;
@@ -1113,7 +1105,7 @@ public class ConnectionTable  {
    * wait for the given incoming connections to receive at least the associated
    * number of messages
    */
-  protected void waitForThreadOwnedOrderedConnectionState(DistributedMember member,
+  protected void waitForThreadOwnedOrderedConnectionState(Stub member,
       Map connectionStates) throws InterruptedException {
     if (Thread.interrupted()) throw new InterruptedException(); // wisest to do this before the synchronize below
     List r = null;
@@ -1123,14 +1115,14 @@ public class ConnectionTable  {
     for (Iterator it=r.iterator(); it.hasNext();) {
       Connection con = (Connection)it.next();
       if (!con.stopped && !con.isClosing() && !con.getOriginatedHere() && con.getPreserveOrder()
-          && member.equals(con.getRemoteAddress())) {
+          && member.equals(con.getRemoteId())) {
         Long state = (Long)connectionStates.remove(Long.valueOf(con.getUniqueId()));
         if (state != null) {
           long count = state.longValue();
           while (!con.stopped && !con.isClosing() && con.getMessagesReceived() < count) {
             if (logger.isDebugEnabled()) {
               logger.debug("Waiting for connection {}/{} currently={} need={}", 
-                  con.getRemoteAddress(), con.getUniqueId(), con.getMessagesReceived(), count);
+                  con.getRemoteId(), con.getUniqueId(), con.getMessagesReceived(), count);
             }
             Thread.sleep(100);
           }
@@ -1238,11 +1230,11 @@ public class ConnectionTable  {
     /**
      * the stub we are connecting to
      */
-    private final DistributedMember id;
+    private final Stub id;
     
     private final Thread connectingThread;
     
-    public PendingConnection(boolean preserveOrder, DistributedMember id) {
+    public PendingConnection(boolean preserveOrder, Stub id) {
       this.preserveOrder = preserveOrder;
       this.id = id;
       this.connectingThread = Thread.currentThread();
@@ -1287,9 +1279,10 @@ public class ConnectionTable  {
 
       boolean severeAlertIssued = false;
       boolean suspected = false;
-      DistributedMember targetMember = null;
+      InternalDistributedMember targetMember = null;
       if (ackSATimeout > 0) {
-        targetMember = this.id;
+        targetMember =
+          ((GMSMembershipManager)mgr).getMemberForStub(this.id, false);
       }
 
       for (;;) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
index a954814..5cd426f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
@@ -18,7 +18,6 @@
 package com.gemstone.gemfire.internal.tcp;
 
 import com.gemstone.gemfire.GemFireException;
-import com.gemstone.gemfire.distributed.DistributedMember;
 
 /**
  * MemberShunnedException may be thrown to prevent ack-ing a message
@@ -29,13 +28,13 @@ import com.gemstone.gemfire.distributed.DistributedMember;
 public class MemberShunnedException extends GemFireException
 {
   private static final long serialVersionUID = -8453126202477831557L;
-  private DistributedMember member;
+  private Stub member;
   
   /**
    * constructor
    * @param member the member that was shunned
    */
-  public MemberShunnedException(DistributedMember member) {
+  public MemberShunnedException(Stub member) {
     super("");
     this.member = member;
   }
@@ -43,7 +42,7 @@ public class MemberShunnedException extends GemFireException
   /**
    * @return the member that was shunned
    */
-  public DistributedMember getShunnedMember() {
+  public Stub getShunnedMember() {
     return this.member;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
index cd711e7..fd495d9 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
@@ -16,7 +16,6 @@
  */
 package com.gemstone.gemfire.internal.tcp;
 
-import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.membership.*;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
@@ -35,7 +34,7 @@ import com.gemstone.gemfire.i18n.LogWriterI18n;
 public interface ServerDelegate {
 
   public void receive( DistributionMessage message, int bytesRead,
-                       DistributedMember connId );
+                       Stub connId );
 
   public LogWriterI18n getLogger();
 
@@ -43,5 +42,5 @@ public interface ServerDelegate {
    * Called when a possibly new member is detected by receiving a direct channel
    * message from him.
    */
-  public void newMemberConnected(InternalDistributedMember member);
+  public void newMemberConnected(InternalDistributedMember member, Stub id);
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java
new file mode 100644
index 0000000..2e4b91b
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.tcp;
+
+import java.io.*;
+import java.net.*;
+
+import com.gemstone.gemfire.DataSerializable;
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
+
+/** Stub represents an ip address and port.
+
+    @author Bruce Schuchardt
+    @since 2.0
+   
+ */
+
+public class Stub implements Externalizable, DataSerializable
+{
+  private InetAddress inAddr;
+  private int port;
+  private int viewID;
+
+  public Stub() {
+    // public default needed for deserialization
+  }
+  
+  public Stub(InetAddress addr, int port, int vmViewID) {
+    viewID = vmViewID;
+    inAddr = addr;
+    this.port = port;
+  }
+  
+  @Override
+  public boolean equals(Object o) {
+    if (o == this) {
+      return true;
+    }
+    if (o instanceof Stub) {
+      Stub s = (Stub)o;
+      boolean result;
+      if (inAddr == null)
+        result = s.inAddr == null;
+      else
+        result = inAddr.equals(s.inAddr);
+      result = result && port == s.port;
+      if (this.viewID != 0 && s.viewID != 0) {
+        result = result && (this.viewID == s.viewID);
+      }
+      return result;
+    }
+    else {
+      return false;
+    }
+  }
+  
+  // hashCode equates to the address hashCode for fast connection lookup
+  @Override
+  public int hashCode() {
+    // do not use viewID in hashCode because it is changed after creating a stub
+    int result = 0;
+    // result += inAddr.hashCode(); // useless
+    result += port;
+    return result;
+  }
+  
+  public void setViewID(int viewID) {
+    this.viewID = viewID;
+  }
+  
+  public int getPort() {
+    return port;
+  }
+  
+  public int getViewID() {
+    return this.viewID;
+  }
+  
+  public InetAddress getInetAddress() {
+    return inAddr;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(80);
+    sb.append("tcp://");
+    if (inAddr == null)
+      sb.append("<null>");
+    else
+      sb.append(inAddr.toString());
+    if (this.viewID != 0) {
+      sb.append("<v"+this.viewID+">");
+    }
+    sb.append(":" + port);
+    return sb.toString();
+  }
+  
+  /**
+   * Writes the contents of this <code>Stub</code> to a
+   * <code>DataOutput</code>. 
+   *
+   * @since 3.0
+   */
+  public void toData(DataOutput out) 
+    throws IOException
+  {
+    DataSerializer.writeInetAddress(inAddr, out);
+    out.writeInt(port);
+    out.writeInt(viewID);
+  }
+  
+  /**
+   * Reads the contents of this <code>Stub</code> from a
+   * <code>DataOutput</code>. 
+   *
+   * @since 3.0
+   */
+  public void fromData(DataInput in)
+    throws IOException, ClassNotFoundException
+  {
+    inAddr = DataSerializer.readInetAddress(in);
+    this.port = in.readInt();
+    this.viewID = in.readInt();
+  }
+
+  /**
+   * static factory method
+   * @since 5.0.2
+   */
+  public static Stub createFromData(DataInput in)
+    throws IOException, ClassNotFoundException
+  {
+    Stub result = new Stub();
+    InternalDataSerializer.invokeFromData(result, in);
+    return result;
+  }
+  
+  public void writeExternal(ObjectOutput os) 
+    throws IOException
+  {
+    this.toData(os);
+  }
+  
+  public void readExternal(ObjectInput is)
+    throws IOException, ClassNotFoundException
+  {
+    this.fromData(is);
+  }
+}


[37/50] [abbrv] incubator-geode git commit: GEODE_468 - fixed AnalyzeSerializablesJUnitTest failure

Posted by je...@apache.org.
GEODE_468 - fixed AnalyzeSerializablesJUnitTest failure

the recent failure was due to the removal of the Stub class and the
renaming of MissingStubException to ShunnedMemberException.


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

Branch: refs/heads/feature/GEODE-14
Commit: df32ad1c657a7b42efc5d00a407045d282788217
Parents: 7c368e9
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 15 14:39:36 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 15 14:40:55 2015 -0800

----------------------------------------------------------------------
 .../gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/df32ad1c/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 03e82d0..8e5c7fd 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
@@ -249,7 +249,7 @@ com/gemstone/gemfire/distributed/internal/deadlock/GemFireDeadlockDetector$Colle
 com/gemstone/gemfire/distributed/internal/deadlock/LocalLockInfo,true,1,info:com/gemstone/gemfire/internal/concurrent/LI,locatility:java/io/Serializable
 com/gemstone/gemfire/distributed/internal/deadlock/LocalThread,true,1,locality:java/io/Serializable,threadId:long,threadName:java/lang/String,threadStack:java/lang/String
 com/gemstone/gemfire/distributed/internal/deadlock/MessageDependencyMonitor$MessageKey,true,414781046295505260,myId:com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember,processorId:int
-com/gemstone/gemfire/distributed/internal/direct/MissingStubException,true,-6455664684151074915
+com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException,true,-6455664684151074915
 com/gemstone/gemfire/distributed/internal/locks/DistributedMemberLock$LockReentryPolicy,false
 com/gemstone/gemfire/distributed/internal/locks/LockGrantorDestroyedException,true,-3540124531032570817
 com/gemstone/gemfire/distributed/internal/membership/gms/messages/InstallViewMessage$messageType,false
@@ -627,7 +627,7 @@ com/gemstone/gemfire/internal/tcp/ByteBufferInputStream,false,buffer:com/gemston
 com/gemstone/gemfire/internal/tcp/ConnectExceptions,true,-4173688946448867706,causes:java/util/List,members:java/util/List
 com/gemstone/gemfire/internal/tcp/ConnectionException,true,-1977443644277412122
 com/gemstone/gemfire/internal/tcp/ImmutableByteBufferInputStream,false
-com/gemstone/gemfire/internal/tcp/MemberShunnedException,true,-8453126202477831557,member:com/gemstone/gemfire/internal/tcp/Stub
+com/gemstone/gemfire/internal/tcp/MemberShunnedException,true,-8453126202477831557,member:com/gemstone/gemfire/distributed/DistributedMember
 com/gemstone/gemfire/internal/tcp/ReenteredConnectException,false
 com/gemstone/gemfire/internal/tcp/VersionedByteBufferInputStream,false,version:com/gemstone/gemfire/internal/Version
 com/gemstone/gemfire/internal/util/Breadcrumbs$CrumbType,false


[28/50] [abbrv] incubator-geode git commit: Revert "Revert "Removing TCPConduit's Stub ID class""

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
index a2801c1..41e5837 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
@@ -45,6 +45,7 @@ import org.apache.logging.log4j.Logger;
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
@@ -63,7 +64,7 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 
 /** <p>TCPConduit manages a server socket and a collection of connections to
-    other systems.  Connections are identified by host/port Stubs.
+    other systems.  Connections are identified by DistributedMember IDs.
     These types of messages are currently supported:</p><pre>
 
        DistributionMessage - message is delivered to the server's
@@ -175,9 +176,8 @@ public class TCPConduit implements Runnable {
   
   ////////////////// runtime state that is re-initialized on a restart
 
-  /** id is an endpoint Stub representing this server.  It holds the
-      actual port the server is listening on */
-  private Stub id;
+  /** server socket address */
+  private InetSocketAddress id;
 
   protected volatile boolean stopped;
 
@@ -351,7 +351,7 @@ public class TCPConduit implements Runnable {
     try {
       localPort = socket.getLocalPort();
 
-      id = new Stub(socket.getInetAddress(), localPort, 0);
+      id = new InetSocketAddress(socket.getInetAddress(), localPort);
       stopped = false;
       ThreadGroup group =
         LoggingThreadGroup.createThreadGroup("P2P Listener Threads", logger);
@@ -371,23 +371,12 @@ public class TCPConduit implements Runnable {
       }
     }
     catch (IOException io) {
-      String s = "While creating ServerSocket and Stub on port " + p;
+      String s = "While creating ServerSocket on port " + p;
       throw new ConnectionException(s, io);
     }
     this.port = localPort;
   }
   
-  /**
-   * After startup we install the view ID into the conduit stub to avoid
-   * confusion during overlapping shutdown/startup from the same member.
-   * 
-   * @param viewID
-   */
-  public void setVmViewID(int viewID) {
-    this.id.setViewID(viewID);
-  }
-
-
   /** creates the server sockets.  This can be used to recreate the
    *  socket using this.port and this.bindAddress, which must be set
    *  before invoking this method.
@@ -596,7 +585,7 @@ public class TCPConduit implements Runnable {
   public void run() {
     ConnectionTable.threadWantsSharedResources();
     if (logger.isTraceEnabled(LogMarker.DM)) {
-      logger.trace(LogMarker.DM, "Starting P2P Listener on  {}", this.getId());
+      logger.trace(LogMarker.DM, "Starting P2P Listener on  {}", id);
     }
     for(;;) {
       SystemFailure.checkFailure();
@@ -731,7 +720,7 @@ public class TCPConduit implements Runnable {
     } // for
 
     if (logger.isTraceEnabled(LogMarker.DM)) {
-      logger.debug("Stopped P2P Listener on  {}", this.getId());
+      logger.debug("Stopped P2P Listener on  {}", id);
     }
   }
 
@@ -807,7 +796,7 @@ public class TCPConduit implements Runnable {
    * @since 5.1
    */
   public void getThreadOwnedOrderedConnectionState(
-    Stub member,
+    DistributedMember member,
     Map result)
   {
     getConTable().getThreadOwnedOrderedConnectionState(member, result);
@@ -819,7 +808,7 @@ public class TCPConduit implements Runnable {
    * with the key
    * @since 5.1
    */
-  public void waitForThreadOwnedOrderedConnectionState(Stub member, Map channelState)
+  public void waitForThreadOwnedOrderedConnectionState(DistributedMember member, Map channelState)
     throws InterruptedException
   {
     // if (Thread.interrupted()) throw new InterruptedException(); not necessary done in waitForThreadOwnedOrderedConnectionState
@@ -842,13 +831,12 @@ public class TCPConduit implements Runnable {
       msg.setBytesRead(bytesRead);
       msg.setSender(receiver.getRemoteAddress());
       msg.setSharedReceiver(receiver.isSharedResource());
-      directChannel.receive(msg, bytesRead, receiver.getRemoteId());
+      directChannel.receive(msg, bytesRead);
     }
   }
 
-  /** gets the Stub representing this conduit's ServerSocket endpoint.  This
-      is used to generate other stubs containing endpoint information. */
-  public Stub getId() {
+  /** gets the address of this conduit's ServerSocket endpoint */
+  public InetSocketAddress getId() {
     return id;
   }
 
@@ -870,21 +858,16 @@ public class TCPConduit implements Runnable {
   }
 
   
-  /** gets the channel that is used to process non-Stub messages */
+  /** gets the channel that is used to process non-DistributedMember messages */
   public DirectChannel getDirectChannel() {
     return directChannel;
   }
 
-  public InternalDistributedMember getMemberForStub(Stub s, boolean validate) {
-    return membershipManager.getMemberForStub(s, validate);
-  }
-  
   public void setLocalAddr(InternalDistributedMember addr) {
     localAddr = addr;
-    this.id.setViewID(addr.getVmViewId());
   }
   
-  public InternalDistributedMember getLocalId() {
+  public InternalDistributedMember getLocalAddr() {
     return localAddr;
   }
 
@@ -894,7 +877,6 @@ public class TCPConduit implements Runnable {
    * member is in the membership view and the system is not shutting down.
    * 
    * @param memberAddress the IDS associated with the remoteId
-   * @param remoteId the TCPConduit stub for this member
    * @param preserveOrder whether this is an ordered or unordered connection
    * @param retry false if this is the first attempt
    * @param startTime the time this operation started
@@ -902,8 +884,8 @@ public class TCPConduit implements Runnable {
    * @param ackSATimeout the ack-severe-alert-threshold * 1000 for the operation to be transmitted (or zero)
    * @return the connection
    */
-  public Connection getConnection(InternalDistributedMember memberAddress, Stub remoteId, final boolean preserveOrder, boolean retry,
-      long startTime, long ackTimeout, long ackSATimeout)
+  public Connection getConnection(InternalDistributedMember memberAddress, final boolean preserveOrder, boolean retry, long startTime,
+      long ackTimeout, long ackSATimeout)
     throws java.io.IOException, DistributedSystemDisconnectedException
   {
     //final boolean preserveOrder = (processorType == DistributionManager.SERIAL_EXECUTOR )|| (processorType == DistributionManager.PARTITIONED_REGION_EXECUTOR);
@@ -922,11 +904,7 @@ public class TCPConduit implements Runnable {
       // problems.  Tear down the connection so that it gets
       // rebuilt.
       if (retry || conn != null) { // not first time in loop
-        // Consult with the membership manager; if member has gone away,
-        // there will not be an entry for this stub.
-        InternalDistributedMember m = this.membershipManager.getMemberForStub(remoteId, true);
-        if (m == null) {
-          // OK, the member left.  Just register an error.
+        if (!membershipManager.memberExists(memberAddress) || membershipManager.isShunned(memberAddress) || membershipManager.shutdownInProgress()) {
           throw new IOException(LocalizedStrings.TCPConduit_TCPIP_CONNECTION_LOST_AND_MEMBER_IS_NOT_IN_VIEW.toLocalizedString());
         }
         // bug35953: Member is still in view; we MUST NOT give up!
@@ -941,15 +919,14 @@ public class TCPConduit implements Runnable {
         }
         
         // try again after sleep
-        m = this.membershipManager.getMemberForStub(remoteId, true);
-        if (m == null) {
+        if (!membershipManager.memberExists(memberAddress) || membershipManager.isShunned(memberAddress)) {
           // OK, the member left.  Just register an error.
           throw new IOException(LocalizedStrings.TCPConduit_TCPIP_CONNECTION_LOST_AND_MEMBER_IS_NOT_IN_VIEW.toLocalizedString());
         }
         
         // Print a warning (once)
         if (memberInTrouble == null) {
-          memberInTrouble = m;
+          memberInTrouble = memberAddress;
           logger.warn(LocalizedMessage.create(LocalizedStrings.TCPConduit_ATTEMPTING_TCPIP_RECONNECT_TO__0, memberInTrouble));
         }
         else {
@@ -963,8 +940,8 @@ public class TCPConduit implements Runnable {
         if (conn != null) {
           try { 
             if (logger.isDebugEnabled()) {
-              logger.debug("Closing old connection.  conn={} before retrying. remoteID={} memberInTrouble={}",
-                  conn, remoteId,  memberInTrouble);
+              logger.debug("Closing old connection.  conn={} before retrying. memberInTrouble={}",
+                  conn, memberInTrouble);
             }
             conn.closeForReconnect("closing before retrying"); 
           } 
@@ -985,10 +962,10 @@ public class TCPConduit implements Runnable {
         boolean debugRetry = false;
         do {
           retryForOldConnection = false;
-          conn = getConTable().get(remoteId, preserveOrder, startTime, ackTimeout, ackSATimeout);
+          conn = getConTable().get(memberAddress, preserveOrder, startTime, ackTimeout, ackSATimeout);
           if (conn == null) {
             // conduit may be closed - otherwise an ioexception would be thrown
-            problem = new IOException(LocalizedStrings.TCPConduit_UNABLE_TO_RECONNECT_TO_SERVER_POSSIBLE_SHUTDOWN_0.toLocalizedString(remoteId));
+            problem = new IOException(LocalizedStrings.TCPConduit_UNABLE_TO_RECONNECT_TO_SERVER_POSSIBLE_SHUTDOWN_0.toLocalizedString(memberAddress));
           } else if (conn.isClosing() || !conn.getRemoteAddress().equals(memberAddress)) {
             if (logger.isDebugEnabled()) {
               logger.debug("Got an old connection for {}: {}@{}", memberAddress, conn, conn.hashCode());
@@ -1027,15 +1004,14 @@ public class TCPConduit implements Runnable {
 
       if (problem != null) {
         // Some problems are not recoverable; check and error out early.
-        InternalDistributedMember m = this.membershipManager.getMemberForStub(remoteId, true);
-        if (m == null) { // left the view
+        if (!membershipManager.memberExists(memberAddress) || membershipManager.isShunned(memberAddress)) { // left the view
           // Bracket our original warning
           if (memberInTrouble != null) {
             // make this msg info to bracket warning
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.TCPConduit_ENDING_RECONNECT_ATTEMPT_BECAUSE_0_HAS_DISAPPEARED, memberInTrouble));
           }
-          throw new IOException(LocalizedStrings.TCPConduit_PEER_HAS_DISAPPEARED_FROM_VIEW.toLocalizedString(remoteId));
+          throw new IOException(LocalizedStrings.TCPConduit_PEER_HAS_DISAPPEARED_FROM_VIEW.toLocalizedString(memberAddress));
         } // left the view
 
         if (membershipManager.shutdownInProgress()) { // shutdown in progress
@@ -1054,12 +1030,12 @@ public class TCPConduit implements Runnable {
         if (memberInTrouble == null) {
           logger.warn(LocalizedMessage.create(
           LocalizedStrings.TCPConduit_ERROR_SENDING_MESSAGE_TO_0_WILL_REATTEMPT_1,
-          new Object[] {m, problem}));
-          memberInTrouble = m;
+          new Object[] {memberAddress, problem}));
+          memberInTrouble = memberAddress;
         }
         else {
           if (logger.isDebugEnabled()) {
-            logger.debug("Error sending message to {}", m, problem);
+            logger.debug("Error sending message to {}", memberAddress, problem);
           }
         }
 
@@ -1073,7 +1049,7 @@ public class TCPConduit implements Runnable {
             throw (IOException)problem;
           }
           else {
-            IOException ioe = new IOException( LocalizedStrings.TCPConduit_PROBLEM_CONNECTING_TO_0.toLocalizedString(remoteId));
+            IOException ioe = new IOException( LocalizedStrings.TCPConduit_PROBLEM_CONNECTING_TO_0.toLocalizedString(memberAddress));
             ioe.initCause(problem);
             throw ioe;
           }
@@ -1089,8 +1065,8 @@ public class TCPConduit implements Runnable {
             LocalizedStrings.TCPConduit_SUCCESSFULLY_RECONNECTED_TO_MEMBER_0,
             memberInTrouble));
         if (logger.isTraceEnabled()) {
-          logger.trace("new connection is {} remoteId={} memberAddress={}", conn, remoteId, memberAddress);
-      }
+          logger.trace("new connection is {} memberAddress={}", conn, memberAddress);
+        }
       }
       return conn;
       }
@@ -1102,180 +1078,6 @@ public class TCPConduit implements Runnable {
     } // for(;;)
   }
 
-//   /**
-//    * Send a message.
-//    * @return the connection used to send the message
-//    * @throws IOException if peer departed view or shutdown in progress
-//    */
-//   private Connection send(Stub remoteId, ByteBuffer bb, boolean preserveOrder, DistributionMessage msg)
-//     throws java.io.IOException
-//   {
-//     if (stopped) {
-//       throw new ConduitStoppedException("The conduit is stopped");
-//     }
-
-//     if (!QUIET) {
-//       LogWriterI18n l = getLogger();
-//       if (l.finerEnabled()) {
-//         l.finer(id.toString() + " sending " + bb
-//                 + " to " + remoteId);
-//       }
-//     }
-
-//     Connection conn = null;
-//     InternalDistributedMember memberInTrouble = null;
-//     for (;;) {
-//       // If this is the second time through this loop, we had
-//       // problems.  Tear down the connection so that it gets
-//       // rebuilt.
-//       if (conn != null) { // not first time in loop
-//         // Consult with the membership manager; if member has gone away,
-//         // there will not be an entry for this stub.
-//         InternalDistributedMember m = membershipManager.getMemberForStub(remoteId);
-//         if (m == null) {
-//           // OK, the member left.  Just register an error.
-//           throw new IOException("TCP/IP connection lost and member no longer in view");
-//         }
-//         // bug35953: Member is still in view; we MUST NOT give up!
-        
-//         // Pause just a tiny bit...
-//         try {
-//           Thread.sleep(5000);
-//         }
-//         catch (InterruptedException e) {
-//           Thread.currentThread().interrupt();
-//           if (membershipManager.shutdownInProgress()) { // shutdown in progress
-//             // Bracket our original warning
-//             if (memberInTrouble != null) {
-//               logger.info("Ending retry attempt because shutdown has started.");
-//             }
-//             throw new IOException("Abandoned because shutdown is in progress");
-//           } // shutdown in progress
-          
-//           // Strange random interrupt intercepted?
-//           logger.warning("Thread has been interrupted but no shutdown in progress", e);
-//           throw new DistributedSystemDisconnectedException(e);
-//         }
-        
-//         // Print a warning (once)
-//         if (memberInTrouble == null) {
-//           memberInTrouble = m;
-//           getLogger().warning("Attempting TCP/IP reconnect to " + memberInTrouble); 
-//         }
-//         else {
-//           getLogger().fine("Attempting TCP/IP reconnect to " + memberInTrouble);
-//         }
-        
-//         // Close the connection (it will get rebuilt later).
-//         this.stats.incReconnectAttempts();
-//         try { 
-//           conn.closeForReconnect("closing before retrying"); 
-//           } 
-//         catch (CancelException ex) {
-//           // In general we ignore close problems, but if the system
-//           // is shutting down, we should just quit.
-//           throw ex;
-//         }
-//         catch (Exception ex) {
-//           }
-//       } // not first time in loop
-      
-//       // Do the send
-//       Exception problem = null;
-//       try {
-//         // Get (or regenerate) the connection
-//         // bug36202: this could generate a ConnectionException, so it
-//         // must be caught and retried
-//         conn = getConTable().get(remoteId, preserveOrder);
-//         //      getLogger().info ("connections returned " + conn);
-//         if (conn == null) {
-//           // conduit may be closed - otherwise an ioexception would be thrown
-//           throw new IOException("Unable to reconnect to server; possible shutdown: " 
-//               + remoteId);
-//         }
-
-//         conn.sendPreserialized(bb, msg);
-//       }
-//       catch (ConnectionException e) {
-//         // Race condition between acquiring the connection and attempting
-//         // to use it: another thread closed it.
-//         problem = e;
-//       }
-//       catch (IOException e) {
-//         problem = e;
-//       }
-
-//       if (problem != null) {
-//         // Some problems are not recoverable; check an error out early.
-//         InternalDistributedMember m = membershipManager.getMemberForStub(remoteId);
-//         if (m == null) { // left the view
-//           // Bracket our original warning
-//           if (memberInTrouble != null) {
-//             logger.info("Ending retry attempt because " + memberInTrouble 
-//                 + " has disappeared.");
-//           }
-//           throw new IOException("Peer has disappeared from view");
-//         } // left the view
-        
-//         if (membershipManager.shutdownInProgress()) { // shutdown in progress
-//           // Bracket our original warning
-//           if (memberInTrouble != null) {
-//             logger.info("Ending retry attempt because shutdown has started.");
-//           }
-//           throw new IOException("Abandoned because shutdown is in progress");
-//         } // shutdown in progress
-        
-//         if (endpointRemoved(remoteId)) { // endpoint removed
-//           // TODO what does this mean?
-//           // Bracket our original warning
-//           if (memberInTrouble != null) {
-//             logger.info("Ending retry attempt because " + memberInTrouble 
-//                 + " has lost its endpoint.");
-//           }
-//           throw new IOException("Endpoint was removed");
-//         } // endpoint removed
-        
-//         // Log the warning.  We wait until now, because we want
-//         // to have m defined for a nice message...
-//         if (memberInTrouble == null) {
-//           logger.warning(
-//               "Error sending message to " + m + " (will reattempt): " 
-//                   + problem.toString(), 
-//               logger.finerEnabled() ? problem : null);
-//           memberInTrouble = m;
-//         }
-//         else {
-//           logger.fine("Error sending message to " + m, problem);
-//         }
-        
-//         // Retry the operation (indefinitely)
-//         continue;
-//       } // problem != null
-//       // Success!
-      
-//       // Make sure our logging is bracketed if there was a problem
-//       if (memberInTrouble != null) {
-//         logger.info("Successfully reestablished connection to server " 
-//             + memberInTrouble);
-//       }
-//       return conn;
-//     } // while retry
-//   }
-
-//   /**
-//    * Sends an already serialized message in a byte buffer
-//    * to the given endpoint. Waits for the send to complete
-//    * before returning.
-//    * @return the connection used to send the message
-//    */
-//   public Connection sendSync(Stub remoteId, ByteBuffer bb, int processorType, DistributionMessage msg)
-//     throws java.io.IOException
-//   {
-//     return send(remoteId, bb,
-//                 processorType == DistributionManager.SERIAL_EXECUTOR,
-//                 msg);
-//   }
-
   @Override
   public String toString() {
     return "" + id;
@@ -1301,22 +1103,22 @@ public class TCPConduit implements Runnable {
     return directChannel.getDM();
   }
   /**
-   * Closes any connections used to communicate with the given stub
+   * Closes any connections used to communicate with the given member
    */
-  public void removeEndpoint(Stub stub, String reason) {
-    removeEndpoint(stub, reason, true);
+  public void removeEndpoint(DistributedMember mbr, String reason) {
+    removeEndpoint(mbr, reason, true);
   }
   
-  public void removeEndpoint(Stub stub, String reason, boolean notifyDisconnect) {
+  public void removeEndpoint(DistributedMember mbr, String reason, boolean notifyDisconnect) {
     ConnectionTable ct = this.conTable;
     if (ct == null) {
       return;
     }
-    ct.removeEndpoint(stub, reason, notifyDisconnect);
+    ct.removeEndpoint(mbr, reason, notifyDisconnect);
   }
   
   /** check to see if there are still any receiver threads for the given end-point */
-  public boolean hasReceiversFor(Stub endPoint) {
+  public boolean hasReceiversFor(DistributedMember endPoint) {
     ConnectionTable ct = this.conTable;
     return (ct != null) && ct.hasReceiversFor(endPoint);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
index 1f411bb..773ef38 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
@@ -38,7 +38,6 @@ 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.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
@@ -46,13 +45,10 @@ import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.distributed.internal.membership.gms.MembershipManagerHelper;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
-import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.tcp.Stub;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
-import dunit.SerializableCallable;
 import dunit.SerializableRunnable;
 import dunit.VM;
 
@@ -192,7 +188,7 @@ public class DistributionManagerDUnitTest extends DistributedTestCase {
       sys.getLogWriter().info("<ExpectedException action=add>attempt to add old member</ExpectedException>");
       sys.getLogWriter().info("<ExpectedException action=add>Removing shunned GemFire node</ExpectedException>");
       try {
-        boolean accepted = mgr.addSurpriseMember(mbr, new Stub());
+        boolean accepted = mgr.addSurpriseMember(mbr);
         Assert.assertTrue("member with old ID was not rejected (bug #44566)", !accepted);
       } finally {
         sys.getLogWriter().info("<ExpectedException action=remove>attempt to add old member</ExpectedException>");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
index 44e1b46..ddbda0b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
@@ -16,8 +16,18 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.mgr;
 
-import static org.mockito.Mockito.*;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -36,7 +46,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
-import com.gemstone.gemfire.distributed.internal.AdminMessageType;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -44,7 +53,6 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.HighPriorityAckedMessage;
-import com.gemstone.gemfire.distributed.internal.HighPriorityDistributionMessage;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.MembershipListener;
 import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
@@ -60,19 +68,11 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.SuspectMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Authenticator;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.HealthMonitor;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.JoinLeave;
-import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Messenger;
-import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
-import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager.StartupEvent;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.admin.remote.AdminRequest;
-import com.gemstone.gemfire.internal.admin.remote.AdminResponse;
 import com.gemstone.gemfire.internal.admin.remote.AlertListenerMessage;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.internal.tcp.ConnectExceptions;
-import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -295,13 +295,6 @@ public class GMSMembershipManagerJUnitTest {
     suspectMember = mockMembers[1];
     manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember, "testing"));
     verify(listener).memberSuspect(suspectMember, mockMembers[0], "testing");
-    
-    InternalDistributedMember mbr = manager.getMemberForStub(new Stub(myMemberId.getInetAddress(), 2033, 20), false);
-    assertTrue(mbr == null);
-    myMemberId.setDirectChannelPort(2033);
-    mbr = manager.getMemberForStub(new Stub(myMemberId.getInetAddress(), 2033, 20), false);
-    assertTrue(mbr != null);
-    assertEquals(mbr, myMemberId);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
index 78c462f..ffd5092 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
@@ -22,6 +22,7 @@ import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.verify;
 
 import java.io.InputStream;
+import java.net.InetSocketAddress;
 import java.net.Socket;
 
 import org.junit.Test;
@@ -61,7 +62,7 @@ public class ConnectionJUnitTest {
     when(stopper.cancelInProgress()).thenReturn(null);
     when(conduit.getCancelCriterion()).thenReturn(stopper);
 
-    when(conduit.getId()).thenReturn(new Stub(SocketCreator.getLocalHost(), 10337, 1));
+    when(conduit.getId()).thenReturn(new InetSocketAddress(SocketCreator.getLocalHost(), 10337));
     
     // NIO can't be mocked because SocketChannel has a final method that
     // is used by Connection - configureBlocking


[40/50] [abbrv] incubator-geode git commit: GEODE-653: Add unit test for GMSHealthMonitor checkIfAvailable Removed unused code Minor javadoc corrections

Posted by je...@apache.org.
GEODE-653: Add unit test for GMSHealthMonitor checkIfAvailable
Removed unused code
Minor javadoc corrections


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

Branch: refs/heads/feature/GEODE-14
Commit: 8c9af2ab4f4a03eda607fe3d6c95511e7d181117
Parents: d40d8a7
Author: Jason Huynh <hu...@gmail.com>
Authored: Fri Dec 11 10:17:39 2015 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Dec 15 16:07:33 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/fd/GMSHealthMonitor.java     | 114 +------------------
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |  42 ++++---
 2 files changed, 31 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c9af2ab/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
index 2aac935..33c7e76 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
@@ -86,12 +86,12 @@ import com.gemstone.gemfire.internal.Version;
  * to remove that member from view.
  * 
  * It has {@link #suspect(InternalDistributedMember, String)} api, which can be used
- * to initiate suspect processing for any member. First is checks whether member is
- * responding or not. Then it informs to probable coordinators to remove that member from
+ * to initiate suspect processing for any member. First is checks whether the member is
+ * responding or not. Then it informs probable coordinators to remove that member from
  * view.
  * 
  * It has {@link #checkIfAvailable(DistributedMember, String, boolean)} api to see
- * if that member is alive. Then based on removal flag it initiate the suspect processing
+ * if that member is alive. Then based on removal flag it initiates the suspect processing
  * for that member.
  * 
  * */
@@ -158,9 +158,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
 
   private ExecutorService checkExecutor;
 
-//  List<SuspectRequest> suspectRequests = new ArrayList<SuspectRequest>();
-//  private RequestCollector<SuspectRequest> suspectRequestCollectorThread;
-
   /**
    * to stop check scheduler
    */
@@ -464,17 +461,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
   }
 
   /**
-   * Check for recent messaging activity from the given member
-   * @param suspectMember
-   * @return whether there has been activity within memberTimeout ms
-   */
-  private boolean checkRecentActivity(InternalDistributedMember suspectMember) {
-    TimeStamp ts = memberTimeStamps.get(suspectMember);
-    return (ts != null && (System.currentTimeMillis() - ts.getTime()) <= memberTimeout);
-  }
-  
-
-  /**
    * During final check, establish TCP connection between current member and suspect member.
    * And exchange PING/PONG message to see if the suspect member is still alive.
    * 
@@ -1238,100 +1224,6 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
     return this.socketPort;
   }
 
-  interface Callback<T> {
-    public void process(List<T> requests);
-  }
-
-  /***
-   * this thread will collect suspect message for some time interval
-   * then it send message to current coordinator first if its not in
-   * suspected list. if its in then it will send message to next probable
-   * coordinator. NOTE: this thread will not check-server for verification
-   * assuming many servers are going down and lets coordinator deals with it.
-   * 
-   * Should we wait for ack from coordinator/probable coordinator that I got
-   * request to suspect these members.
-   * 
-   */
-  class RequestCollector<T> extends Thread {
-    volatile boolean shutdown = false;
-    final List<T> listToTrack;
-    final Callback<T> callback;
-    final long timeout;
-
-    public RequestCollector(String name, ThreadGroup tg, List<T> l, Callback<T> c, long t) {
-      super(tg, name);
-      listToTrack = l;
-      callback = c;
-      timeout = t;
-    }
-
-    void shutdown() {
-      shutdown = true;
-      synchronized (listToTrack) {
-        listToTrack.notify();
-        interrupt();
-      }
-    }
-
-    boolean isShutdown() {
-      return shutdown;
-    }
-
-    @Override
-    public void run() {
-      List<T> requests = null;
-      logger.debug("Suspect thread is starting");
-      long okayToSendSuspectRequest = System.currentTimeMillis() + timeout;
-      try {
-        for (;;) {
-          synchronized (listToTrack) {
-            if (shutdown || services.getCancelCriterion().isCancelInProgress()) {
-              return;
-            }
-            if (listToTrack.isEmpty()) {
-              try {
-                logger.trace("Result collector is waiting");
-                listToTrack.wait();
-              } catch (InterruptedException e) {
-                return;
-              }
-            } else {
-              long now = System.currentTimeMillis();
-              if (now < okayToSendSuspectRequest) {
-                // sleep to let more suspect requests arrive
-                try {
-                  sleep(okayToSendSuspectRequest - now);
-                  continue;
-                } catch (InterruptedException e) {
-                  return;
-                }
-              } else {
-                if (requests == null) {
-                  requests = new ArrayList<T>(listToTrack);
-                } else {
-                  requests.addAll(listToTrack);
-                }
-                listToTrack.clear();
-                okayToSendSuspectRequest = System.currentTimeMillis() + timeout;
-              }
-            }
-          } // synchronized
-          if (requests != null && !requests.isEmpty()) {
-            if (logger != null && logger.isDebugEnabled()) {
-              logger.info("Health Monitor is sending {} member suspect requests to coordinator", requests.size());
-            }
-            callback.process(requests);
-            requests = null;
-          }
-        }
-      } finally {
-        shutdown = true;
-        logger.debug("Suspect thread is stopped");
-      }
-    }
-  }
-
   private void sendSuspectRequest(final List<SuspectRequest> requests) {
     // the background suspect-collector thread is currently disabled
 //    synchronized (suspectRequests) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c9af2ab/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
index eb17ca8..d539374 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
@@ -20,7 +20,11 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -42,6 +46,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
@@ -407,26 +413,34 @@ public class GMSHealthMonitorJUnitTest {
    * validates HealthMonitor.CheckIfAvailable api
    */
   @Test
-  public void testCheckIfAvailable() {
-
-    NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());
-
-    // 3rd is current member
-    when(messenger.getMemberID()).thenReturn(mockMembers.get(3));
-
-    gmsHealthMonitor.installView(v);
-
+  public void testCheckIfAvailableNoHeartBeatDontRemoveMember() {
     long startTime = System.currentTimeMillis();
-
     boolean retVal = gmsHealthMonitor.checkIfAvailable(mockMembers.get(1), "Not responding", false);
-
     long timeTaken = System.currentTimeMillis() - startTime;
 
-    assertTrue("This should have taken member ping timeout 100ms ", timeTaken > 90);
-    assertTrue("CheckIfAvailable should have return false", !retVal);
+    assertTrue("This should have taken member ping timeout 100ms ", timeTaken >= gmsHealthMonitor.memberTimeout);
+    assertFalse("CheckIfAvailable should have return false", retVal);
   }
 
   @Test
+  public void testCheckIfAvailableWithSimulatedHeartBeat() {
+    InternalDistributedMember memberToCheck = mockMembers.get(1);
+    HeartbeatMessage fakeHeartbeat = new HeartbeatMessage();
+    fakeHeartbeat.setSender(memberToCheck);
+    when(messenger.send(any(HeartbeatRequestMessage.class))).then(new Answer() {
+      @Override
+      public Object answer(InvocationOnMock invocation) throws Throwable {
+        gmsHealthMonitor.processMessage(fakeHeartbeat);
+        return null;
+      }
+    });
+    
+    boolean retVal = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
+    assertTrue("CheckIfAvailable should have return true", retVal);
+  }
+  
+  
+  @Test
   public void testShutdown() {
 
     NetView v = new NetView(mockMembers.get(0), 2, mockMembers, new HashSet<InternalDistributedMember>(), new HashSet<InternalDistributedMember>());


[34/50] [abbrv] incubator-geode git commit: GEODE-669 Add CacheObserver for ShutdownAll

Posted by je...@apache.org.
GEODE-669
Add CacheObserver for ShutdownAll

Let GatewayReceiverCommand to throw CacheClosedException instead of
RegionDestroyedException when ShutdownAll happened.


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

Branch: refs/heads/feature/GEODE-14
Commit: ae2d529405f28caa0ab9fc1a3d309ebddbda4920
Parents: 0b288a2
Author: zhouxh <gz...@pivotal.io>
Authored: Tue Dec 15 11:21:32 2015 -0800
Committer: zhouxh <gz...@pivotal.io>
Committed: Tue Dec 15 11:22:56 2015 -0800

----------------------------------------------------------------------
 .../gemfire/internal/cache/CacheObserver.java   |  5 ++++
 .../internal/cache/CacheObserverAdapter.java    |  4 ++++
 .../internal/cache/GemFireCacheImpl.java        |  7 ++++++
 .../sockets/command/GatewayReceiverCommand.java | 24 +++++++++++++-------
 4 files changed, 32 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ae2d5294/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserver.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserver.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserver.java
index 9cd6c28..6b18d86 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserver.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserver.java
@@ -183,4 +183,9 @@ public interface CacheObserver
    */
   public void beforeDeletingEmptyOplog(Oplog emptyOplog);
   
+  /**
+   * Invoked just before ShutdownAll operation
+   * @param emptyOplog
+   */
+  void beforeShutdownAll();
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ae2d5294/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserverAdapter.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserverAdapter.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserverAdapter.java
index 82c4507..ef66739 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserverAdapter.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheObserverAdapter.java
@@ -147,4 +147,8 @@ public class CacheObserverAdapter implements CacheObserver {
   public void beforeDeletingEmptyOplog(Oplog emptyOplog)
   {
   }
+  
+  @Override
+  public void beforeShutdownAll() {
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ae2d5294/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 27bb813..506bd7a 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
@@ -1639,6 +1639,13 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
       // it's already doing shutdown by another thread
       return;
     }
+    if (LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER) {
+      try {
+        CacheObserverHolder.getInstance().beforeShutdownAll();
+      } finally {
+        LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
+      }
+    }
     this.isShutDownAll = true;
 
     // bug 44031 requires multithread shutdownall should be grouped

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ae2d5294/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
index 6d252e4..e2fb686 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.EntryNotFoundException;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.operations.DestroyOperationContext;
@@ -36,6 +37,7 @@ import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
@@ -67,6 +69,16 @@ public class GatewayReceiverCommand extends BaseCommand {
   private GatewayReceiverCommand() {
   }
 
+  private void handleRegionNull(ServerConnection servConn, String regionName, int batchId) {
+    GemFireCacheImpl gfc = (GemFireCacheImpl)servConn.getCachedRegionHelper().getCache();
+    if (gfc != null && gfc.isCacheAtShutdownAll()) {
+      throw new CacheClosedException("Shutdown occurred during message processing");
+    } else {
+      String reason = LocalizedStrings.ProcessBatch_WAS_NOT_FOUND_DURING_BATCH_CREATE_REQUEST_0.toLocalizedString(new Object[] {regionName, Integer.valueOf(batchId)});
+      throw new RegionDestroyedException(reason, regionName);
+    }
+  }
+  
   @Override
   public void cmdExecute(Message msg, ServerConnection servConn, long start)
       throws IOException, InterruptedException {
@@ -292,8 +304,7 @@ public class GatewayReceiverCommand extends BaseCommand {
           }
           region = (LocalRegion)crHelper.getRegion(regionName);
           if (region == null) {
-            String reason = LocalizedStrings.ProcessBatch_WAS_NOT_FOUND_DURING_BATCH_CREATE_REQUEST_0.toLocalizedString(new Object[] {regionName, Integer.valueOf(batchId)});
-            throw new RegionDestroyedException(reason, regionName);
+            handleRegionNull(servConn, regionName, batchId);
           } else {
             clientEvent = new EntryEventImpl(eventId);
             if (versionTimeStamp > 0) {
@@ -402,8 +413,7 @@ public class GatewayReceiverCommand extends BaseCommand {
           }
           region = (LocalRegion)crHelper.getRegion(regionName);
           if (region == null) {
-            String reason = LocalizedStrings.ProcessBatch_WAS_NOT_FOUND_DURING_BATCH_CREATE_REQUEST_0.toLocalizedString(new Object[] {regionName, Integer.valueOf(batchId)});
-            throw new RegionDestroyedException(reason, regionName);
+            handleRegionNull(servConn, regionName, batchId);
           } else {
             clientEvent = new EntryEventImpl(eventId);
             if (versionTimeStamp > 0) {
@@ -502,8 +512,7 @@ public class GatewayReceiverCommand extends BaseCommand {
           }
           region = (LocalRegion)crHelper.getRegion(regionName);
           if (region == null) {
-            String reason = LocalizedStrings.ProcessBatch_WAS_NOT_FOUND_DURING_BATCH_CREATE_REQUEST_0.toLocalizedString(new Object[] {regionName, Integer.valueOf(batchId)});
-            throw new RegionDestroyedException(reason, regionName);
+            handleRegionNull(servConn, regionName, batchId);
           } else {
             clientEvent = new EntryEventImpl(eventId);
             if (versionTimeStamp > 0) {
@@ -584,8 +593,7 @@ public class GatewayReceiverCommand extends BaseCommand {
             region = (LocalRegion)crHelper.getRegion(regionName);
             
             if (region == null) {
-              String reason = LocalizedStrings.ProcessBatch_WAS_NOT_FOUND_DURING_BATCH_UPDATE_VERSION_REQUEST_0.toLocalizedString(new Object[] {regionName});
-              throw new RegionDestroyedException(reason, regionName);
+              handleRegionNull(servConn, regionName, batchId);
             } else {
 
               clientEvent = new EntryEventImpl(eventId);


[21/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/css/usergrid-site.css
----------------------------------------------------------------------
diff --git a/gemfire-site/content/css/usergrid-site.css b/gemfire-site/content/css/usergrid-site.css
deleted file mode 100644
index 88bcc48..0000000
--- a/gemfire-site/content/css/usergrid-site.css
+++ /dev/null
@@ -1,1554 +0,0 @@
-@import url("http://fonts.googleapis.com/css?family=Open+Sans:400,300,600,800,700,400italic,600italic,700italic,800italic,300italic");
-/* @import url("http://fonts.googleapis.com/css?family=Source+Sans+Pro:200,300,400,700"); */
-/* Key scaffolding
--------------------------------------------------- */
-body {
-  position: relative; /* For scrollyspy */
-  padding-top: 50px; /* Account for fixed navbar */
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif !important;
-  font-size:14px;
-  background-color: #FAFBFC;
-  /*font-family: "Source Sans Pro", "Helvetica Neue", Helvetica, Arial, sans-serif; */
-}
-h1, h2, h3, h4, .h1, .h2, .h3, .h4 {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif !important;
-  font-weight: 300;
-}
-a:hover {text-decoration: none;}
-h2>a, h3>a, h2>a:hover, h3>a:hover { color: rgb(51,51,51); }
-h2:hover>a:after {
-  content: "\F0C1";
-  font-family: 'FontAwesome';
-  /*position:absolute;*/
-  padding-left: 12px;
-  vertical-align: 15%;
-  font-size: 20px;
-}
-h3:hover>a:after {
-  content: "\F0C1";
-  font-family: 'FontAwesome';
-  /*position:absolute;*/
-  padding-left: 8px;
-  vertical-align: 15%;
-  font-size: 15px;
-}
-h5, h6, .h5, .h6 {
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif !important;
-}
-.bf-sidebar .nav-list code{
-  margin-left: -3px;
-}
-.done {
-  opacity: 0.6;
-}
-.nav-list > li {
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 400 !important;
-  opacity: 0.8;
-  margin-bottom: 3px;
-}
-header .navbar-nav {
-  font-size: 120%;
-}
-.nav-list > li > ul {
-  padding-left: 0;
-}
-.nav-list > li ul ul {
-  padding-left: 19px;
-}
-.nav-list > li ul > li {
-  list-style-type: none !important;
-}
-.nav-list > li ul > li:before {
-  content:"\221F";
-  font-weight: 200;
-  opacity: 0.5;
-  line-height: 50%;
-  vertical-align: 60%;
-  padding-left: 3px;
-}
-.nav-list > li ul > li > a {
-  padding: 0 0 0 3px;
-  margin-bottom: 0px;
-}
-.nav-list > li > a {
-  padding: 0;
-}
-.nav-list > li > a.icns-apple:before {
-  text-align: right;
-  content: "\F179";
-  font-family: 'FontAwesome';
-  font-size: 110%;
-  margin-left: -18px;
-  padding-right: 8px;
-  line-height: 0;
-}
-.nav-list > li > a.icns-android:before {
-  text-align: right;
-  content: "\F17B";
-  font-family: 'FontAwesome';
-  font-size: 110%;
-  margin-left: -18px;
-  padding-right: 8px;
-  line-height: 0;
-}
-.nav-list > li > a.icns-html5:before {
-  text-align: right;
-  content: "\F13B";
-  font-family: 'FontAwesome';
-  font-size: 110%;
-  margin-left: -18px;
-  padding-right: 8px;
-  line-height: 0;
-}
-.nav-list > li > a.icns-windows:before {
-  text-align: right;
-  content: "\F17A";
-  font-family: 'FontAwesome';
-  margin-left: -19px;
-  padding-right: 7px;
-  line-height: 0;
-}
-li.nav-header {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 600 !important;
-  opacity: 1.0;
-  padding-top: 0 !important;
-  padding-left: 0 !important;
-  font-size: 120%;
-  margin-top: 33px;
-}
-.twitter-timeline {
-  margin-top: 33px;
-}
-.bf-docs-container .nav-list li.nav-header {
-  padding: 0 !important;
-  border-bottom: solid 1px #ccc;
-  line-height: 0;
-  margin-top: 33px;
-  margin-bottom: 13px;
-}
-.bf-docs-container .nav-list li.nav-header span {
-  background-color: #FAFBFC;
-  padding-right: 8px;
-}
-
-.bf-community h2, .bf-docs-container h2, .bf-docs-container h3 {
-  padding-top: 90px;
-  margin-top: -60px;
-}
-
-.bf-docs-container .github-callout {
-  padding: 20px 0 40px 0;
-  font-style: italic;
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  opacity: 0.8;
-  font-size: 110%;
-}
-.navbar-nav > li a {
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 300 !important;
-  opacity: 0.8;
-}
-
-.navbar-nav > li > a.major {
-  font-weight: 400 !important;
-  opacity: 1.0;
-}
-
-.navbar-collapse {
-    max-height: none;
-}
-
-.bf-masthead-bg p {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-style: normal;
-  font-weight: 200;
-  font-size: 120%;
-  line-height: 120%;
-}
-.bf-features p {
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-style: normal;
-  font-weight: 400;
-}
-.bf-features p a {
-  /*font-weight: 600;*/
-  /*color: rgb(52,73,94);*/
-}
-a.btn {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 500 !important;
-  font-style: normal;
-  letter-spacing: 110%;
-}
-h1 {
-  font-weight: 200;
-  letter-spacing: 110%;
-  font-size: 200%;
-}
-.page-header {
-  padding-top: 2px;
-}
-.page-header h1 {
-  padding-top:20px;
-  font-weight: 400;
-  font-size:350%;
-}
-.page-header small{
-  float: right;
-  background-color: #fff;
-  padding-left: 10px;
-}
-.page-header small a {
-  color: #ddd;
-  font-style: italic;
-}
-.page-header small a:hover {
-  color: #2481A6;
-}
-h2 {
-  font-weight: 400;
-}
-h3 {
-  padding-top: 20px;
-  font-weight: 400 !important;
-}
-.bf-features h3 {
-  padding-top: 0;
-}
-#copyright {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 300 !important;
-  font-style: italic;
-}
-#copyright img {
-  border-radius: 4px;
-}
-#copyright p {
-  opacity: 0.8;
-}
-#copyright .credits {
-  color: #fff;
-  font-weight: 600;
-  opacity: 1 !important;
-  font-style: normal;
-}
-.table {
-  margin: 20px 0 0 0;
-  background-color: transparent;
-}
-.table thead>tr>th, .table tbody>tr>th, .table tfoot>tr>th, .table thead>tr>td, .table tbody>tr>td, .table tfoot>tr>td {
-  padding-left: 0;
-}
-.toc {
-  margin-bottom:40px;
-  /*background-color: #fff;
-  border-radius: 4px;*/
-  /*padding: 20px;*/
-}
-/*.toc > h4 {
-  padding-top: 0;
-  margin-top: 0;
-  font-size: 150%;
-  font-weight:400;
-}*/
-.toc > ul {
-  margin-bottom: 0;
-  padding-left: 20px;
-}
-section > .container > h2 {
-  color: #2481A6 !important;
-}
-div.row > div.text-left > h2 {
-  margin-top: 40px;
-  margin-bottom: 10px;
-}
-.bf-docs-container h2 {
-  font-size: 250%;
-}
-/* font-face
--------------------------------------------------- */
-@font-face {
-  font-family: 'icomoon';
-  src:url('../fonts/icomoon.eot');
-  src:url('../fonts/icomoon.eot?#iefix') format('embedded-opentype'),
-    url('../fonts/icomoon.woff') format('woff'),
-    url('../fonts/icomoon.ttf') format('truetype'),
-    url('../fonts/icomoon.svg#icomoon') format('svg');
-  font-weight: normal;
-  font-style: normal;
-}
-[data-icon]:before {
-  font-family: 'icomoon';
-  content: attr(data-icon);
-  speak: none;
-  font-weight: normal;
-  font-variant: normal;
-  text-transform: none;
-  line-height: 1;
-  -webkit-font-smoothing: antialiased;
-}
-.icon-equalizer, .icon-phone-portrait, .icon-rocket, .icon-github, .icon-twitter, .icon-thumbf-up, .icon-earth, .icon-bug, .icon-wiki, .icon-releases,.icon-window {
-  font-family: 'icomoon';
-  speak: none;
-  font-style: normal;
-  font-weight: normal;
-  font-variant: normal;
-  text-transform: none;
-  line-height: 1;
-  -webkit-font-smoothing: antialiased;
-}
-.icon-equalizer:before {
-  content: "\e000";
-}
-.icon-phone-portrait:before {
-  content: "\e002";
-}
-.icon-rocket:before {
-  content: "\e003";
-}
-.icon-github:before {
-  content: "\e001";
-}
-.icon-twitter:before {
-  content: "\e004";
-}
-.icon-thumbf-up:before {
-  content: "\e005";
-}
-.icon-earth:before {
-  content: "\e006";
-}
-.icon-bug:before {
-  content: "\e007";
-}
-.icon-window:before {
-  content: "\e008";
-}
-.icon-wiki:before {
-  content: "\e009";
-}
-.icon-releases:before {
-  content: "\e010";
-}
-/* Custom docs button */
-.btn-bf {
-  color: #563d7c;
-  background-color: #fff;
-  border-color: #e5e5e5;
-}
-.btn-bf:hover,
-.btn-bf:focus,
-.btn-bf:active {
-  color: #fff;
-  background-color: #563d7c;
-  border-color: #563d7c;
-}
-/* Navs and headers
--------------------------------------------------- */
-header.navbar .container {
-  padding-right: 0;
-}
-footer > .container {
-  padding-right: 0;
-  padding-left: 0;
-}
-header.navbar.secondary {
-  box-shadow: 0px 1px 2px #7b48fe;
-}
-.navbar-inverse {
-    background-color: #3c3c3c;
-    /*background-color: #3c3c3c;*/
-}
-nav.navbar-collapse {
-  border-color: rgb(52, 73, 94) !important;
-  border-width: 1px;
-  box-shadow: none;
-  padding-left: 15px;
-}
-.bf-docs-nav {
-  border-bottom: 0px;
-  padding-left: 20px;
-  padding-right: 20px;
-}
-.navbar-nav {
-  margin: 0;
-}
-
-.navbar-brand {
-  padding-left: 0;
-}
-.bf-docs-nav .navbar-nav > li > a {
-  margin: 10px 0 0 10px;
-  padding: 5px;
-  -webkit-border-radius: 4px;
-  -moz-border-radius: 4px;
-  border-radius: 4px;
-  color: white;
-}
-.bf-docs-nav .navbar-nav > li > a:hover {
-  color: #fff;
-  background-color: #1e90ff;
-}
-.bf-docs-nav .navbar-nav > .active > a,
-.bf-docs-nav .navbar-nav > .active > a:hover {
-  color: #fff;
-  background-color: #1e90ff !important;
-}
-li.divider {
-  margin-left: 0 !important;
-}
-
-/* Homepage masthead
--------------------------------------------------- */
-.bf-masthead {
-  position: relative;
-  text-align: left;
-  /*background: #cbcbcd url(../img/rende.png) no-repeat 50% 100px;*/
-}
-.bf-masthead .bf-masthead-bg {
-    /*margin-top: 10px;*/
-    padding: 30px 15px;
-    color:#3c3c3c;
-    background-color: #fff;
-}
-/*.bf-masthead .logo {
-  height: 203px;
-}*/
-.bf-masthead .logo-title {
-  width: 600px;
-  padding: 20px 0 0 0;
-  margin-left: 0;
-}
-.bf-masthead .text-container {
-  margin-left: auto;
-  margin-right: auto;
-  text-align: left;
-}
-.bf-masthead .tagline {
-  font-style:normal;
-  font-weight:200;
-  font-size: 300%;
-  margin-bottom: 0;
-  margin-top: 30px;
-  padding-bottom: 30px;
-  color: #3c3c3c;
-  line-height: 90%;
-}
-.bf-masthead .tagline em {
-  font-style: italic;
-}
-.bf-masthead .description {
-  font-style: normal;
-  font-size: 150%;
-  margin-bottom: 45px;
-  /*line-height: 100%;*/
-}
-.bf-masthead .btn-wrapper {
-  text-align: center;
-  margin-bottom: 20px;
-  clear:both;
-}
-.bf-masthead .btn-wrapper {
-  text-align: left;
-  width: 100%;
-  margin-right: 20px;
-}
-.bf-masthead .social-btn {
-  float: left;
-  margin: 0 20px 10px 0;
-}
-.bf-masthead .or {
-  margin-left: 0px;
-  font-style: normal;
-  font-weight: 200;
-  opacity: 0.8;
-}
-
-.bf-masthead a {
-  line-height: 30px;
-  color: #FAFBFC;
-  font-style: normal;
-  font-weight: 400;
-}
-.bf-masthead h1 {
-  font-size: 30px;
-  line-height: 1.4;
-  color: #fff;
-}
-.bf-masthead .bf-masthead-links a,
-.bf-masthead .bf-masthead-links a:hover {
-  color: #fff;
-}
-.bf-masthead-info {
-  padding: 80px 20px;
-  font-size: 28px;
-  text-align: center;
-  background-color: #fc5d7b;
-  color: rgba(0, 0, 0, .3)
-}
-.bf-masthead-info .icon-angle-down {
-  font-size: 40px;
-}
-
-/* Download button */
-.bf-masthead .btn-outline {
-  margin-top: 20px;
-  margin-bottom: 20px;
-  border: 1px solid #fff;
-  padding: 18px 24px;
-  font-size: 21px;
-  color: #fff;
-}
-.bf-masthead .btn-outline:hover {
-  background-color: #fff;
-  color: #42a8d2;
-}
-
-.bf-masthead-links {
-  margin-top: 20px;
-  margin-bottom: 20px;
-  padding-left: 0;
-  list-style: none;
-  text-align: center;
-}
-.bf-masthead-links li {
-  margin-bottom: 15px;
-  color: #fff;
-}
-.bf-masthead-links a {
-  color: #c8c7c6;
-}
-
-/* Homepage features
--------------------------------------------------- */
-section > .container > h2 {
-  text-align: left;
-  margin-bottom: 16px;
-  padding-bottom: 6px;
-  font-size: 28px;
-  color:#3c3c3c;
-  /* border-bottom: 1px solid #e5e5e5; */
-}
-.bf-features {
-  color: white;
-  text-align: center;
-  padding: 20px 0;
-}
-.bf-features h2 ~ p {
-  margin-bottom: 40px;
-  font-size: 16px;
-}
-.bf-features .bf-icon-wrap {
-  margin: 0 auto 10px;
-  -webkit-border-radius: 50%;
-  -moz-border-radius: 50%;
-  border-radius: 50%;
-  background-color: #7b48fe;
-  width: 90px;
-  height: 90px;
-  line-height: 90px;
-  font-size: 50px;
-  color: #FAFBFC;
-}
-.bf-features h3 {
-  margin: 0 0 10px;
-  color: rgb(52,73,94);
-  font-weight: 400 !important;
-}
-.bf-features h3 ~ p {
-  color: #656565;
-}
-/*
-.bf-features2 {
-  color: #292929;
-  text-align: center;
-}
-.bf-features2 {
-  padding: 40px 0;
-}
-.bf-features2 h2 {
-  text-align: left;
-  margin-bottom: 16px;
-  padding-bottom: 6px;
-  font-size: 28px;
-  color:#666666;
-  border-bottom: 1px solid #e5e5e5;
-}
-.bf-features2 h2 ~ p {
-  margin-bottom: 40px;
-  font-size: 16px;
-}
-.bf-features2 .bf-icon-wrap {
-  margin: 0 auto 20px;
-  -webkit-border-radius: 50%;
-  -moz-border-radius: 50%;
-  border-radius: 50%;
-  background-color: #e5e9ed;
-  width: 80px;
-  height: 80px;
-  line-height: 80px;
-  font-size: 40px;
-  color: #a5b6c7;
-}
-.bf-features2 h3 {
-  margin: 0 0 10px;
-  color: #696969;
-}
-.bf-features2 h3 ~ p {
-  color: #656565;
-} */
-.bf-features-inverse {
-  padding: 140px 0;
-  font-size: 16px;
-  color: white;
-  text-align: center;
-  background-color: #3c3c3c;
-}
-.bf-features-inverse h2 {
-  margin-bottom: 20px;
-  font-size: 40px;
-}
-.bf-features-inverse h2 ~ p {
-  margin-bottom: 40px;
-  font-size: 16px;
-}
-.bf-features-inverse .bf-icon-wrap {
-  margin: 0 auto 20px;
-  -webkit-border-radius: 50%;
-  -moz-border-radius: 50%;
-  border-radius: 50%;
-  background-color: #fff;
-  width: 80px;
-  height: 80px;
-  line-height: 80px;
-  font-size: 40px;
-  color: #3dc0f1;
-}
-.bf-features-inverse h3 ~ p {
-  font-size: 14px;
-}
-.bf-features-inverse a,
-.bf-features-inverse a:hover {
-  color: #fff;
-  text-decoration: underline;
-}
-.bf-news ul {
-  margin-bottom: 20px;
-}
-
-.bf-questions .container {
-  border-top: 1px solid rgb(52,73,94);
-  border-bottom: 1px solid rgb(52,73,94);
-  padding: 60px 0;
-  margin-top: 0px;
-}
-
-.bf-questions div {
-  line-height: 30px;
-}
-
-.bf-community .container, .bf-tagline .container {
-  padding-left: 0;
-  padding-right: 0;
-}
-
-.bf-community h3 {
-  font-weight: 600;
-  margin-top: 0;
-}
-
-.bf-community .row a p {
-  color: rgb(51,51,51);
-}
-
-/*.bf-community h2 {
-  padding-bottom: 10px;
-  border-bottom: 1px solid rgb(51,51,51);
-}*/
-
-.bf-community h2 {
-  /*padding: 0 !important;*/
-  border-bottom: solid 1px #ccc;
-  line-height: 0;
-  /*margin-top: 33px;*/
-  margin-bottom: 13px;
-}
-.bf-releases h2 {
-  /*padding: 0 !important;*/
-  border-bottom: solid 1px #ccc;
-  line-height: 0;
-  /*margin-top: 33px;*/
-  margin-bottom: 13px;
-}
-
-.bf-community h2 span {
-  background-color: #FAFBFC;
-  padding-right: 10px;
-}
-
-
-.bf-releases h2 span {
-  background-color: #FAFBFC;
-  padding-right: 10px;
-}
-
-.bf-releases p {
-  padding: 10px 0 5px 0;
-}
-
-.bf-community {
-  margin-top: 80px;
-}
-
-.bf-releases {
-  margin-top: 10px;
-}
-
-#map-canvas {
-  border-bottom: 1px solid rgb(52, 73, 94);
-}
-
-.bf-community img {
-  border-radius: 4px;
-  border: solid 1px rgb(52, 73, 94);
-  margin-top: 20px;
-  margin-bottom: 10px;
-  width: 100%;
-  box-shadow: rgb(52, 73, 94) 2px 2px 4px;
-}
-
-.bf-releases img {
-  border-radius: 4px;
-  border: solid 1px rgb(52, 73, 94);
-  margin-top: 20px;
-  margin-bottom: 10px;
-  width: 100%;
-  box-shadow: rgb(52, 73, 94) 2px 2px 4px;
-}
-
-
-h2.icns-calendar:before{
-  text-align: right;
-  content: "\F073";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -40px;
-  padding-right: 10px;
-  line-height: 0;
-}
-h2.icns-chat:before{
-  text-align: right;
-  content: "\F0FC";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -40px;
-  padding-right: 10px;
-  line-height: 0;
-}
-
-h2.icns-envelope:before{
-  text-align: right;
-  content: "\F0E0";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -40px;
-  padding-right: 10px;
-  line-height: 0;
-}
-h2.icns-briefcase:before{
-  text-align: right;
-  content: "\F0B1";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -42px;
-  padding-right: 10px;
-  line-height: 0;
-}
-h2.icns-group:before{
-  text-align: right;
-  content: "\F005";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -37px;
-  padding-right: 8px;
-  line-height: 0;
-}
-h2.icns-tools:before{
-  text-align: right;
-  content: "\F006";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -37px;
-  padding-right: 8px;
-  line-height: 0;
-}
-/* Docs pages and sections
--------------------------------------------------- */
-
-/* Page headers */
-.bf-header {
-  padding: 5px 20px 10px;
-  font-size: 16px;
-  color: #5a5a5a;
-  text-align: center;
-  border-bottom: 1px solid #e5e5e5;
-}
-.bf-header p {
-  font-weight: 300;
-  line-height: 1.5;
-}
-.bf-header .container {
-  position: relative;
-}
-
-.bf-docs-container {
-  margin-top: 60px;
-}
-
-.bf-docs-container .col-md-3{
-  padding-left: 0;
-}
-
-
-.bf-docs-container .col-md-9.main-article {
-  background-color: #fff;
-  border-radius: 4px;
-  border: 1px solid #ccc;
-  padding-bottom: 15px;
-  min-height: 820px;
-}
-
-.bf-docs-section + .bf-docs-section {
-  padding-top: 20px;
-}
-
-.bf-docs-container .nav-list {
-  padding-top: 20px;
-}
-
-/* Bootstrap code examples
--------------------------------------------------- */
-
-/* Base class */
-.bf-example {
-  position: relative;
-  margin: 20px 0 40px;
-}
-.bf-example > .btn-toolbar + .btn-toolbar {
-  margin-top: 10px;
-}
-
-.bf-example .row {
-  margin-bottom: 20px;
-}
-
-/* Tweak display of the examples */
-.bf-example + .highlight {
-  margin: 0 -15px;
-  border-radius: 0;
-  border-width: 0 0 1px;
-}
-
-/* Tweak content of examples for optimum awesome */
-.bf-example > p:last-child,
-.bf-example > ul:last-child,
-.bf-example > ol:last-child,
-.bf-example > blockquote:last-child,
-.bf-example > input:last-child,
-.bf-example > select:last-child,
-.bf-example > textarea:last-child,
-.bf-example > .table:last-child,
-.bf-example > .navbar:last-child
-.bf-example > .jumbotron:last-child,
-.bf-example > .alert:last-child,
-.bf-example > .panel:last-child,
-.bf-example > .list-group:last-child,
-.bf-example > .well:last-child {
-  margin-bottom: 0;
-}
-.bf-example > p > .close {
-  float: none;
-}
-.bf-example-title {
-  border-left: 3px solid #08d;
-  padding-left: 10px;
-  line-height: 24px;
-  margin: 20px 0;
-  font-size: 16px;
-}
-.bf-example-align .btn {
-  margin-top: 4px;
-}
-.bf-example-align .btn-xs {
-  margin-top: 16px;
-}
-.bf-example-align .btn-sm {
-  margin-top: 8px;
-}
-.bf-example-align .btn-lg {
-  margin-top: -7px;
-}
-
-/* Navbar examples */
-.bf-example .navbar:last-child {
-  margin-bottom: 0;
-}
-.bf-navbar-top-example,
-.bf-navbar-bottom-example {
-  z-index: 1;
-  padding: 0;
-  min-height: 110px;
-  overflow: hidden; /* cut the drop shadows off */
-}
-.bf-navbar-top-example .navbar-fixed-top,
-.bf-navbar-bottom-example .navbar-fixed-bottom {
-  position: relative;
-  margin-left: 0;
-  margin-right: 0;
-}
-.bf-navbar-top-example .navbar-fixed-top {
-  top: -1px;
-}
-.bf-navbar-bottom-example .navbar-fixed-bottom {
-  bottom: -1px;
-}
-.bf-navbar-top-example {
-  -webkit-border-radius: 0 0 4px 4px;
-     -moz-border-radius: 0 0 4px 4px;
-          border-radius: 0 0 4px 4px;
-}
-.bf-navbar-top-example:after {
-  top: auto;
-  bottom: 15px;
-  -webkit-border-radius: 0 4px 0 4px;
-     -moz-border-radius: 0 4px 0 4px;
-          border-radius: 0 4px 0 4px;
-}
-.bf-navbar-bottom-example {
-  -webkit-border-radius: 4px 4px 0 0;
-     -moz-border-radius: 4px 4px 0 0;
-          border-radius: 4px 4px 0 0;
-}
-.bf-navbar-bottom-example .navbar {
-  margin-bottom: 0;
-}
-.bf-navbar-top-example.bf-example:after {
-  bottom: -1px;
-  border-radius: 0 4px;
-}
-
-/* Example modals */
-.bf-example-modal {
-  background-color: #f5f5f5;
-}
-.bf-example-modal .modal {
-  position: relative;
-  top: auto;
-  right: auto;
-  left: auto;
-  bottom: auto;
-  z-index: 1;
-  display: block;
-}
-.bf-example-modal .modal-dialog {
-  left: auto;
-  margin-left: auto;
-  margin-right: auto;
-}
-
-/* Example dropdowns */
-.bf-example > .dropdown > .dropdown-menu,
-.bf-example-submenu > .pull-left > .dropup > .dropdown-menu,
-.bf-example-submenu > .pull-left > .dropdown > .dropdown-menu {
-  position: static;
-  display: block;
-  margin-bottom: 5px;
-}
-.bf-example-submenu {
-  min-height: 230px;
-}
-.bf-example-submenu > .pull-left + .pull-left {
-  margin-left: 20px;
-}
-
-/* Tooltips */
-.bf-example-tooltips {
-  text-align: center;
-}
-
-/* Popovers */
-.bf-example-popover {
-  padding-bottom: 24px;
-}
-.bf-example-popover .popover {
-  position: relative;
-  display: block;
-  float: left;
-  width: 260px;
-  margin: 20px;
-}
-
-/* Example templates
--------------------------------------------------- */
-
-.bf-examples h4 {
-  margin-bottom: 5px;
-}
-.bf-examples p {
-  margin-bottom: 20px;
-}
-
-/* Responsive docs
--------------------------------------------------- */
-
-/* Responsive (scrollable) doc tables */
-@media (max-width: 768px) {
-  .bf-table-scrollable {
-    width: 100%;
-    margin-bottom: 15px;
-    overflow-y: hidden;
-    overflow-x: scroll;
-    border: 1px solid #ddd;
-  }
-  .bf-table-scrollable .table {
-    margin-bottom: 0;
-    border: 0;
-  }
-  .bf-table-scrollable .table th,
-  .bf-table-scrollable .table td {
-    white-space: nowrap;
-  }
-  .bf-table-scrollable .table th:first-child,
-  .bf-table-scrollable .table td:first-child {
-    border-left: 0;
-  }
-  .bf-table-scrollable .table th:last-child,
-  .bf-table-scrollable .table td:last-child {
-    border-right: 0;
-  }
-  .bf-table-scrollable .table tr:last-child th,
-  .bf-table-scrollable .table tr:last-child td {
-    border-bottom: 0;
-  }
-}
-
-/* Footer
--------------------------------------------------- */
-
-.bf-footer {
-  margin-top: 40px;
-  padding: 30px 5px;
-  /*text-align: left;*/
-  background:#1e90ff;
-  color:white;
-}
-
-.bf-footer a {
-    color:#f0f0f0 !important;
-}
-.bf-footer.bf-footer-margin {
-    margin-top: 100px;
-    margin-bottom: 20px;
-}
-
-.bf-footer .bf-icon-wrap {
-    font-size: 40px;
-    color: #a5b6c7;
-    display: inline;
-    margin: 10px;
-}
-
-#footer-icons {
-    float: right;
-    display: inline;
-    line-height: 20px;
-    margin: 40px 10px 20px 10px;
-}
-
-#copyright {
-    float: left;
-    display: inline;
-    line-height: 20px;
-    margin: 20px 10px 20px 10px;
-}
-
-/* Social proof buttons from GitHub & Twitter */
-.bf-social {
-  margin-bottom: 20px;
-}
-.bf-social-buttons {
-  display: inline-block;
-  margin-bottom: 0;
-  padding-left: 0;
-  list-style: none;
-}
-.bf-social-buttons li {
-  display: inline-block;
-  line-height: 1;
-}
-.bf-social-buttons li + li {
-  margin-left: 15px;
-}
-.bf-social-buttons .twitter-follow-button {
-  width: 225px !important;
-}
-.bf-social-buttons .twitter-share-button {
-  width: 98px !important;
-}
-/* Style the GitHub buttons via CSS instead of inline attributes */
-.github-btn {
-  border: 0;
-  overflow: hidden;
-}
-
-.highlight {
-  padding: 9px 14px;
-  margin-bottom: 40px;
-  border: 1px solid #e1e1e8;
-  border-radius: 4px;
-}
-.highlight pre {
-  padding: 0;
-  margin-top: 0;
-  margin-bottom: 0;
-  background-color: rgb(248, 248, 255);
-  border: 0;
-  white-space: nowrap;
-}
-.highlight pre code {
-  font-size: inherit;
-  font-family: Consolas, 'Liberation Mono', Courier, monospace;
-  color: #333; /* Effectively the base text color */
-}
-.highlight pre .lineno {
-  display: inline-block;
-  width: 22px;
-  padding-right: 5px;
-  margin-right: 10px;
-  text-align: right;
-  color: #bebec5;
-}
-
-.c { color: #999988; font-style: italic } /* Comment */
-.err { color: #a61717; background-color: #e3d2d2 } /* Error */
-.k { color: #000000; font-weight: bold } /* Keyword */
-.o { color: #000000; font-weight: bold } /* Operator */
-.cm { color: #999988; font-style: italic } /* Comment.Multiline */
-.cp { color: #999999; font-weight: bold; font-style: italic } /* Comment.Preproc */
-.c1 { color: #999988; font-style: italic } /* Comment.Single */
-.cs { color: #999999; font-weight: bold; font-style: italic } /* Comment.Special */
-.gd { color: #000000; background-color: #ffdddd } /* Generic.Deleted */
-.ge { color: #000000; font-style: italic } /* Generic.Emph */
-.gr { color: #aa0000 } /* Generic.Error */
-.gh { color: #999999 } /* Generic.Heading */
-.gi { color: #000000; background-color: #ddffdd } /* Generic.Inserted */
-.go { color: #888888 } /* Generic.Output */
-.gp { color: #555555 } /* Generic.Prompt */
-.gs { font-weight: bold } /* Generic.Strong */
-.gu { color: #aaaaaa } /* Generic.Subheading */
-.gt { color: #aa0000 } /* Generic.Traceback */
-.kc { color: #000000; font-weight: bold } /* Keyword.Constant */
-.kd { color: #000000; font-weight: bold } /* Keyword.Declaration */
-.kn { color: #000000; font-weight: bold } /* Keyword.Namespace */
-.kp { color: #000000; font-weight: bold } /* Keyword.Pseudo */
-.kr { color: #000000; font-weight: bold } /* Keyword.Reserved */
-.kt { color: #445588; font-weight: bold } /* Keyword.Type */
-.m { color: #009999 } /* Literal.Number */
-.s { color: #d01040 } /* Literal.String */
-.na { color: #008080 } /* Name.Attribute */
-.nb { color: #0086B3 } /* Name.Builtin */
-.nc { color: #445588; font-weight: bold } /* Name.Class */
-.no { color: #008080 } /* Name.Constant */
-.nd { color: #3c5d5d; font-weight: bold } /* Name.Decorator */
-.ni { color: #800080 } /* Name.Entity */
-.ne { color: #990000; font-weight: bold } /* Name.Exception */
-.nf { color: #990000; font-weight: bold } /* Name.Function */
-.nl { color: #990000; font-weight: bold } /* Name.Label */
-.nn { color: #555555 } /* Name.Namespace */
-.nt { color: #000080 } /* Name.Tag */
-.nv { color: #008080 } /* Name.Variable */
-.ow { color: #000000; font-weight: bold } /* Operator.Word */
-.w { color: #bbbbbb } /* Text.Whitespace */
-.mf { color: #009999 } /* Literal.Number.Float */
-.mh { color: #009999 } /* Literal.Number.Hex */
-.mi { color: #009999 } /* Literal.Number.Integer */
-.mo { color: #009999 } /* Literal.Number.Oct */
-.sb { color: #d01040 } /* Literal.String.Backtick */
-.sc { color: #d01040 } /* Literal.String.Char */
-.sd { color: #d01040 } /* Literal.String.Doc */
-.s2 { color: #d01040 } /* Literal.String.Double */
-.se { color: #d01040 } /* Literal.String.Escape */
-.sh { color: #d01040 } /* Literal.String.Heredoc */
-.si { color: #d01040 } /* Literal.String.Interpol */
-.sx { color: #d01040 } /* Literal.String.Other */
-.sr { color: #009926 } /* Literal.String.Regex */
-.s1 { color: #d01040 } /* Literal.String.Single */
-.ss { color: #990073 } /* Literal.String.Symbol */
-.bp { color: #999999 } /* Name.Builtin.Pseudo */
-.vc { color: #008080 } /* Name.Variable.Class */
-.vg { color: #008080 } /* Name.Variable.Global */
-.vi { color: #008080 } /* Name.Variable.Instance */
-.il { color: #009999 } /* Literal.Number.Integer.Long */
-
-.css .o,
-.css .o + .nt,
-.css .nt + .nt { color: #999; }
-/* Show code snippets when we have the space */
-@media screen and (min-width: 481px) {
-  .highlight {
-    display: block;
-  }
-}
-
-.demoColorSwatches {
-    color: #FFF;
-}
-.demoColorSwatches h4 {
-    color: #292929;
-}
-.demoColorSwatches .colorPanel {
-    padding: 15px;
-}
-.demoColorSwatches .colorPanel:hover {
-    opacity: 0.85;
-    filter: alpha(opacity=85);
-}
-
-/* Responsive variations
--------------------------------------------------- */
-
-/* Hide code snippets on mobile devices */
-@media screen and (max-width: 480px) {
-  .highlight {
-    display: none;
-  }
-}
-
-
-.bf-docs-container {
-    margin-top: 40px;
-}
-
-/* Navbars */
-.bf-example .navbar:last-child {
-  margin-bottom: 0;
-}
-.bf-navbar-top-example,
-.bf-navbar-bottom-example {
-  z-index: 1;
-  padding: 0;
-  overflow: hidden; /* cut the drop shadows off */
-  border: 1px solid #ddd ;
-}
-.bf-navbar-top-example .navbar-header,
-.bf-navbar-bottom-example .navbar-header {
-  margin-left: 0;
-}
-.bf-navbar-top-example .navbar-fixed-top,
-.bf-navbar-bottom-example .navbar-fixed-bottom {
-  position: relative;
-  margin-left: 0;
-  margin-right: 0;
-}
-.bf-navbar-top-example {
-  padding-bottom: 45px;
-}
-.bf-navbar-top-example:after {
-  top: auto;
-  bottom: 15px;
-}
-.bf-navbar-top-example .navbar-fixed-top {
-  top: -1px;
-}
-.bf-navbar-bottom-example {
-  padding-top: 65px;
-}
-.bf-navbar-bottom-example .navbar-fixed-bottom {
-  bottom: -1px;
-}
-.bf-navbar-bottom-example .navbar {
-  margin-bottom: 0;
-}
-
-/* Phone and below */
-@media screen and (max-width: 768px) {
-  #home-logo { opacity: 1; height: 25px;}
-  span.icon-book:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Docs";
-  }
-  span.icon-group:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Community";
-  }
-  span.icon-github-sign:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Github";
-  }
-  span.icon-trello:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Trello";
-  }
-  span.icon-bug:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " JIRA";
-  }
-  span.icon-stackexchange:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " StackOverflow";
-  }
-  span.icon-chat:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " HipChat";
-  }
-  span.icon-twitter:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Twitter";
-  }
-  span.icon-edit:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: "  Wiki";
-  }
-  span.icon-releases:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Releases";
-  }
-  span.icon-comments:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Chat";
-  }
-  span.icon-envelope:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Mailing Lists";
-  }
-  div.cta {
-    line-height: 60px;
-  }
-  span.avoidwrap { display:inline-block; }
-  .navbar-nav > li > a {
-    opacity: 1.0;
-  }
-  .bf-masthead .tagline {
-    line-height: 90%;
-  }
-
-  .bf-masthead .description {
-    margin-bottom: 30px;
-  }
-  .bf-masthead p {
-    clear:both;
-    line-height: 100%;
-    padding-top: 5px;
-  }
-  .bf-masthead .or {
-    margin-right: 0;
-  }
-  .bf-questions .container{
-    padding: 30px 0;
-  }
-}
-
-/* Phones and Tablets */
-/*@media screen and (min-width: 768px) and (max-width: 992px) {
-
-  .bf-masthead .tagline {
-    padding-bottom: 20px;
-  }
-
-}*/
-
-@media screen and (max-width: 992px) {
-  .bf-community.committers .col-md-3 p{
-    margin-bottom: 40px;
-  }
-}
-
-/* Tablets and below */
-@media screen and (max-width: 1200px) {
-  .row {
-    padding: 0 25px;
-  }
-  .bf-docs-container .nav-list {
-    padding-left: 30px;
-  }
-  .bf-docs-container .colb-md-3 .nav-list {
-    padding-top: 0;
-  }
-  .bf-community .container, .bf-tagline .container {
-    padding-left: 50px;
-  }
-  .bf-community.committers .col-md-3 h3{
-    margin-top: 0px;
-  }
-  #map-canvas { height: 300px; margin-top: 5px;}
-}
-/* Tablets and up */
-@media screen and (min-width: 768px) {
-  header.secondary #home-logo { opacity: 1; }
-  #home-logo { opacity: 0; height: 40px;}
-  .bf-header {
-    font-size: 21px;
-    text-align: left;
-  }
-  .bf-header h1 {
-    font-size: 60px;
-    line-height: 1;
-  }
-
-  .bf-masthead p {
-    float:left;
-  }
-
-  .bf-masthead .or {
-    margin-right: 20px;
-  }
-  .bf-features .row {
-    padding: 20px 0;
-  }
-  .bf-example {
-    margin-left: 0;
-    margin-right: 0;
-    border-radius: 4px;
-    background-color: #fff;
-    border-width: 1px;
-    border-color: #ddd;
-    box-shadow: none;
-  }
-  .bf-example + .prettyprint,
-  .bf-example + .highlight {
-    margin-top: -16px;
-    margin-left: 0;
-    margin-right: 0;
-    border-width: 1px;
-    border-bottom-left-radius: 4px;
-    border-bottom-right-radius: 4px;
-  }
-
-  /* Show the docs nav */
-  .bf-sidebar {
-    display: block;
-  }
-  .bf-sidebar > .nav > li > a {
-    display: inline-block;
-  }
-
-  .bf-navbar-top-example .navbar-fixed-top,
-  .bf-navbar-bottom-example .navbar-fixed-bottom {
-    position: absolute;
-  }
-  .bs-navbar-top-example {
-    border-radius: 0 0 4px 4px;
-  }
-  .bs-navbar-bottom-example {
-    border-radius: 4px 4px 0 0;
-  }
-}
-
-/* Tablets */
-@media screen and (min-width: 768px) and (max-width: 1200px) {
-  span.icns {
-    font-size: 140%;
-  }
-}
-
-/* Tablets/desktops and up */
-@media screen and (min-width: 1200px) {
-  span.icon-book:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Docs";
-  }
-  span.icon-group:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Community";
-  }
-  span.icon-github-sign:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Github";
-  }
-  span.icon-trello:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Trello";
-  }
-  span.icon-bug:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " JIRA";
-  }
-  span.icon-stackexchange:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " StackOverflow";
-  }
-  span.icon-chat:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " HipChat";
-  }
-  span.icon-twitter:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Twitter";
-  }
-  span.icon-edit:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: "  Wiki";
-  }
-  span.icon-releases:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Releases";
-  }
-  span.icon-comments:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Chat";
-  }
-  span.icon-envelope:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Mailing Lists";
-  }
-  .bf-header h1,
-  .bf-header p {
-    margin-right: 380px;
-  }
-  .bf-docs-container {
-    margin-top: 40px;
-  }
-
-  .bf-masthead .tagline, .bf-masthead .description, .bf-masthead .btn-wrapper {
-    margin-left: 120px;
-  }
-  .bf-community.committers .col-md-3 h3{
-    margin-top: 40px;
-  }
-  .bf-community.committers img {
-    width: 100%;
-  }
-  #map-canvas { height: 500px; margin-top: 10px;}
-
-}
-
-.navbar-toggle {
-  background-color: rgb(36,129,166);
-  margin: 12px 0 0 0;
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/favicon.ico
----------------------------------------------------------------------
diff --git a/gemfire-site/content/favicon.ico b/gemfire-site/content/favicon.ico
deleted file mode 100644
index 392c757..0000000
Binary files a/gemfire-site/content/favicon.ico and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/font/FontAwesome.otf
----------------------------------------------------------------------
diff --git a/gemfire-site/content/font/FontAwesome.otf b/gemfire-site/content/font/FontAwesome.otf
deleted file mode 100755
index 7012545..0000000
Binary files a/gemfire-site/content/font/FontAwesome.otf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/font/fontawesome-webfont-eot.eot
----------------------------------------------------------------------
diff --git a/gemfire-site/content/font/fontawesome-webfont-eot.eot b/gemfire-site/content/font/fontawesome-webfont-eot.eot
deleted file mode 100755
index 0662cb9..0000000
Binary files a/gemfire-site/content/font/fontawesome-webfont-eot.eot and /dev/null differ



[32/50] [abbrv] incubator-geode git commit: Previous commit (49ffab7c00d0f261646d8e453fa65bffe0f66aa8) message should be: Merge branch 'feature/GEODE-655' into develop Merge PR #55

Posted by je...@apache.org.
Previous commit (49ffab7c00d0f261646d8e453fa65bffe0f66aa8) message should be: Merge branch 'feature/GEODE-655' into develop Merge PR #55


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

Branch: refs/heads/feature/GEODE-14
Commit: 15b065549ad7cf6cb53668afaa143f9795a9100c
Parents: 49ffab7
Author: William Markito <wm...@pivotal.io>
Authored: Tue Dec 15 09:49:28 2015 -0800
Committer: William Markito <wm...@pivotal.io>
Committed: Tue Dec 15 09:49:28 2015 -0800

----------------------------------------------------------------------

----------------------------------------------------------------------



[03/50] [abbrv] incubator-geode git commit: GEODE-647: fix GfshParserJUnitTest

Posted by je...@apache.org.
GEODE-647: fix GfshParserJUnitTest


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

Branch: refs/heads/feature/GEODE-14
Commit: 80c61f4fd38114591cadc3fa28209206f66a2c3f
Parents: b0419b1
Author: Jens Deppe <jd...@pivotal.io>
Authored: Fri Dec 11 13:33:18 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri Dec 11 13:45:01 2015 -0800

----------------------------------------------------------------------
 .../internal/cli/GfshParserJUnitTest.java       | 50 ++++++++------------
 .../internal/cli/HeadlessGfshJUnitTest.java     |  7 ++-
 2 files changed, 24 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/80c61f4f/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
index 68fe251..f28b5cc 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
@@ -25,7 +25,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.junit.After;
-import org.junit.Ignore;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.springframework.shell.core.CommandMarker;
@@ -134,6 +134,26 @@ public class GfshParserJUnitTest {
   private static final Completion[] OPTION3_COMPLETIONS = {
       new Completion("option3"), new Completion("option3Alternate") };
 
+  private CommandManager commandManager;
+
+  private GfshParser parser;
+
+  @Before
+  public void setUp() throws Exception {
+    // Make sure no prior tests leave the CommandManager in a funky state
+    CommandManager.clearInstance();
+
+    commandManager = CommandManager.getInstance(false);
+    commandManager.add(Commands.class.newInstance());
+    commandManager.add(SimpleConverter.class.newInstance());
+    commandManager.add(StringArrayConverter.class.newInstance());
+    commandManager.add(StringListConverter.class.newInstance());
+    // Set up the parser
+    parser = new GfshParser(commandManager);
+
+    CliUtil.isGfshVM = false;
+  }
+
   @After
   public void tearDown() {
     CommandManager.clearInstance();
@@ -150,14 +170,6 @@ public class GfshParserJUnitTest {
    */
   @Test
   public void testComplete() throws Exception {
-    // get a CommandManager, add sample commands
-    CommandManager commandManager = CommandManager.getInstance(false);
-    assertNotNull("CommandManager should not be null.", commandManager);
-    commandManager.add(Commands.class.newInstance());
-    commandManager.add(SimpleConverter.class.newInstance());
-    // Set up the parser
-    GfshParser parser = new GfshParser(commandManager);
-
     // Get the names of the command
     String[] command1Names = ((CliCommand) METHOD_command1
         .getAnnotation(CliCommand.class)).value();
@@ -454,14 +466,6 @@ public class GfshParserJUnitTest {
    */
   @Test
   public void testCompleteAdvanced() throws Exception {
-    // get a CommandManager, add sample commands
-    CommandManager commandManager = CommandManager.getInstance(false);
-    assertNotNull("CommandManager should not be null.", commandManager);
-    commandManager.add(Commands.class.newInstance());
-    commandManager.add(SimpleConverter.class.newInstance());
-    // Set up the parser
-    GfshParser parser = new GfshParser(commandManager);
-
     // Get the names of the command
     String[] command1Names = ((CliCommand) METHOD_command1
         .getAnnotation(CliCommand.class)).value();
@@ -778,19 +782,7 @@ public class GfshParserJUnitTest {
    * @throws SecurityException
    */
   @Test
-  @Ignore("GEODE-647")
   public void testParse() throws Exception {
-    // get a CommandManager, add sample commands
-    CommandManager commandManager = CommandManager.getInstance(false);
-    assertNotNull("CommandManager should not be null.", commandManager);
-    commandManager.add(StringArrayConverter.class.newInstance());
-    commandManager.add(StringListConverter.class.newInstance());
-    commandManager.add(SimpleConverter.class.newInstance());
-    commandManager.add(Commands.class.newInstance());
-
-    // Set up the parser
-    GfshParser parser = new GfshParser(commandManager);
-
     // Get the names of the command
     String[] command1Names = ((CliCommand) METHOD_command1
         .getAnnotation(CliCommand.class)).value();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/80c61f4f/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfshJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfshJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfshJUnitTest.java
index 0807898..6053494 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfshJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/HeadlessGfshJUnitTest.java
@@ -22,7 +22,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.management.internal.MBeanJMXAdapter;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -38,10 +38,10 @@ import static org.junit.Assert.assertTrue;
  *
  * @author tushark
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class HeadlessGfshJUnitTest {
 
-  @SuppressWarnings({"unused", "deprecation", "unused"})
+  @SuppressWarnings({"deprecation"})
   @Test
   public void testHeadlessGfshTest() throws ClassNotFoundException, IOException, InterruptedException {
     GemFireCacheImpl cache = null;
@@ -57,7 +57,6 @@ public class HeadlessGfshJUnitTest {
 
     ds = DistributedSystem.connect(pr);
     cache = (GemFireCacheImpl) CacheFactory.create(ds);
-    ObjectName name = MBeanJMXAdapter.getDistributedSystemName();
 
     HeadlessGfsh gfsh = new HeadlessGfsh("Test", 25);
     for (int i = 0; i < 5; i++) {


[26/50] [abbrv] incubator-geode git commit: GEODE-650: Fix DiskStoreCOmmandsDUnitTest - previous tests not cleaning up properly

Posted by je...@apache.org.
GEODE-650: Fix DiskStoreCOmmandsDUnitTest - previous tests not cleaning up properly


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

Branch: refs/heads/feature/GEODE-14
Commit: ec9d16a3130e33ee53d837301dc35c3a8bf0e99d
Parents: 7cf253e
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Dec 14 08:25:55 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Dec 14 08:25:55 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ec9d16a3/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
index 6623403..e6272a7 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommandsDUnitTest.java
@@ -81,6 +81,7 @@ public class MemberCommandsDUnitTest extends CacheTestCase {
   @Override
   public void tearDown2() throws Exception {
     super.tearDown2();
+    disconnectFromDS();
     CliUtil.isGfshVM = true;
   }
 


[36/50] [abbrv] incubator-geode git commit: GEODE-667: Update system-rules dependency from 1.12.1 to 1.15.0

Posted by je...@apache.org.
GEODE-667: Update system-rules dependency from 1.12.1 to 1.15.0


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

Branch: refs/heads/feature/GEODE-14
Commit: 7c368e9553bac0ee7d5d31e252912480346f2f39
Parents: c30e533
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Dec 14 09:32:30 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 15 14:07:06 2015 -0800

----------------------------------------------------------------------
 gradle/dependency-versions.properties | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7c368e95/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 684e60e..3833f54 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -79,4 +79,4 @@ stephenc-findbugs.version = 1.3.9-1
 spymemcached.version = 2.9.0
 swagger.version = 1.3.2
 swagger-springmvc.version = 0.8.2
-system-rules.version = 1.12.1
+system-rules.version = 1.15.0


[09/50] [abbrv] incubator-geode git commit: GEODE-291: Prevent stderr noise from com.gemstone.gemfire.test.process

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java
index 2847b34..67904bc 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java
@@ -47,8 +47,7 @@ import com.gemstone.gemfire.internal.logging.LogService;
 public class ProcessWrapper {
   private static final Logger logger = LogService.getLogger();
 
-  protected static final String TIMEOUT_MILLIS_PROPERTY = "process.test.timeoutMillis";
-  protected static final long TIMEOUT_MILLIS_DEFAULT = 5 * 60 * 1000;
+  private static final long PROCESS_TIMEOUT_MILLIS = 10 * 60 * 1000L; // 10 minutes
   private static final long DELAY = 10;
 
   private final boolean headless;
@@ -323,11 +322,12 @@ public class ProcessWrapper {
   
         this.stdout = stdOut;
         this.stderr = stdErr;
-        this.outputReader = new ProcessOutputReader(this.process, stdOut, stdErr, this.allLines);
+        this.outputReader = new ProcessOutputReader(this.process, stdOut, stdErr);
         this.started = true;
       }
       
-      this.outputReader.waitFor();
+      this.outputReader.start();
+      this.outputReader.waitFor(PROCESS_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
       int code = this.process.waitFor();
       
       synchronized (this.exitValue) {
@@ -424,7 +424,7 @@ public class ProcessWrapper {
     private String[] mainArguments = null;
     private boolean useMainLauncher = true;
     private boolean headless = true;
-    private long timeoutMillis = TIMEOUT_MILLIS_DEFAULT;
+    private long timeoutMillis = PROCESS_TIMEOUT_MILLIS;
     private boolean inline = false;
     public Builder() {
       //nothing

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index d02110c..8ea38c3 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -21,9 +21,11 @@ activation.version = 1.1.1
 annotations.version = 3.0.0
 antlr.version = 2.7.7
 asm.version = 5.0.3
-assertj-core.version = 2.1.0
+assertj-core.version = 3.2.0
 awaitility.version = 1.6.5
 bcel.version = 5.2
+catch-exception.version = 1.4.4
+catch-throwable.version = 1.4.4
 cglib.version = 3.1
 classmate.version = 0.9.0
 commons-collections.version = 3.2.1


[23/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/community/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/community/index.html b/gemfire-site/content/community/index.html
deleted file mode 100644
index d1bac78..0000000
--- a/gemfire-site/content/community/index.html
+++ /dev/null
@@ -1,629 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-
-<head>
-    <meta charset="utf-8">
-    <title>Apache Geode (incubating) — </title>
-    <meta http-equiv="x-ua-compatible" content="ie=edge" />
-    <meta name="viewport" content="width=device-width, initial-scale=1, minimum-scale=1, maximum-scale=1" />
-    <meta name="keywords" content="Apache Geode, Geode, GemFire, In-memory, IMDB, IMDG, cache">
-    <meta name="description" content="Apache Geode is a distributed, in-memory database with strong data consistency, built to support transactional applications with low latency and high concurrency needs." />
-    <meta property="og:title" content="Apache Geode (incubating)" />
-    <meta property="og:description" content="Apache Geode is a distributed, in-memory database with strong data consistency, built to support transactional applications with low latency and high concurrency needs." />
-    <!-- Loading Typekit -->
-    <script type="text/javascript" src="//use.typekit.net/ddl7izx.js"></script>
-    <script type="text/javascript">try{Typekit.load();}catch(e){}</script>
-    <!-- Place this tag right after the last button or just before your close body tag. -->
-    <script async defer id="github-bjs" src="https://buttons.github.io/buttons.js"></script>
-        <!-- Loading Bootstrap -->
-    <link href="/bootstrap/bootstrap.min.css" rel="stylesheet" type='text/css'>
-    <link href="/css/bootflat.css" rel="stylesheet" type='text/css'>
-    <link href="/css/geode-site.css" rel="stylesheet" type='text/css'>
-    <link href="https://fonts.googleapis.com/css?family=Open+Sans:200,400,500,300,600,800,700,400italic,600italic,700italic,800italic,300italic" rel="stylesheet" type="text/css">
-    <link href="/css/font-awesome.min.css" rel="stylesheet" type='text/css'>
-
-    
-    <script type="text/javascript"
-      src="https://maps.googleapis.com/maps/api/js?key=AIzaSyBYSxyIKS22zC4wSLFXZGR8QKXbGWLFwYw&sensor=false">
-    </script>
-    <script type="text/javascript">
-
-        var points = [
-                new google.maps.LatLng(-33.8674869,151.2069902),
-                new google.maps.LatLng(40.2338438,-111.6585337),
-                new google.maps.LatLng(37.6909682,-122.3107517),
-                new google.maps.LatLng(-37.6825027,176.1880232),
-                new google.maps.LatLng(-0.023559,37.906193),
-                new google.maps.LatLng(41.00527,28.97696),
-                new google.maps.LatLng(47.1569444,27.5902778),
-                new google.maps.LatLng(49.261226,-123.1139268),
-                new google.maps.LatLng(55.378051,-3.435973),
-                new google.maps.LatLng(30.3321838,-81.65565099999999),
-                new google.maps.LatLng(30.267153,-97.7430608),
-                new google.maps.LatLng(-14.235004,-51.92528),
-                new google.maps.LatLng(41.76371109999999,-72.6850932),
-                new google.maps.LatLng(53.3498053,-6.2603097),
-                new google.maps.LatLng(-37.814107,144.96328),
-                new google.maps.LatLng(26.820553,30.802498),
-                new google.maps.LatLng(34.0522342,-118.2436849),
-                new google.maps.LatLng(37.566535,126.9779692),
-                new google.maps.LatLng(7.873053999999999,80.77179699999999),
-                new google.maps.LatLng(39.737567,-104.9847179),
-                new google.maps.LatLng(52.9399159,-73.5491361),
-                new google.maps.LatLng(40.7143528,-74.00597309999999),
-                new google.maps.LatLng(44.9374831,-93.20099979999999),
-                new google.maps.LatLng(38.963745,35.243322),
-                new google.maps.LatLng(35.7795897,-78.6381787),
-                new google.maps.LatLng(35.907757,127.766922),
-                new google.maps.LatLng(-6.2689913,106.8060388),
-                new google.maps.LatLng(27.3364347,-82.53065269999999),
-                new google.maps.LatLng(44.494887,11.3426163),
-                new google.maps.LatLng(39.952335,-75.16378900000001),
-                new google.maps.LatLng(37.09024,-95.712891),
-                new google.maps.LatLng(40.5852602,-105.084423),
-                new google.maps.LatLng(43.653226,-79.3831843),
-                new google.maps.LatLng(47.6062095,-122.3320708),
-                new google.maps.LatLng(31.046051,34.851612),
-                new google.maps.LatLng(51.41233,-0.300689),
-                new google.maps.LatLng(39.4699075,-0.3762881),
-                new google.maps.LatLng(51.51121389999999,-0.1198244),
-                new google.maps.LatLng(42.3556899,-83.361853),
-                new google.maps.LatLng(33.4483771,-112.0740373),
-                new google.maps.LatLng(45.5234515,-122.6762071),
-                new google.maps.LatLng(43.0730517,-89.4012302),
-                new google.maps.LatLng(36.1031378,-80.202394),
-                new google.maps.LatLng(37.7749295,-122.4194155),
-                new google.maps.LatLng(5.263234100000001,100.4846227),
-                new google.maps.LatLng(35.5950581,-82.5514869),
-                new google.maps.LatLng(35.86166,104.195397),
-                new google.maps.LatLng(34.4208305,-119.6981901),
-                new google.maps.LatLng(1.352083,103.819836),
-                new google.maps.LatLng(36.8507689,-76.28587259999999),
-                new google.maps.LatLng(22.396428,114.109497),
-                new google.maps.LatLng(48.856614,2.3522219),
-                new google.maps.LatLng(40.4167754,-3.7037902),
-                new google.maps.LatLng(18.5204303,73.8567437),
-                new google.maps.LatLng(36.1666667,-86.7833333),
-                new google.maps.LatLng(53.3498053,-6.2603097),
-                new google.maps.LatLng(26.0993883,-80.1343301),
-                new google.maps.LatLng(42.331427,-83.0457538),
-                new google.maps.LatLng(37.82206000000001,-122.272437),
-                new google.maps.LatLng(30.42130899999999,-87.2169149),
-                new google.maps.LatLng(44.4325,26.1038889),
-                new google.maps.LatLng(41.0700485,-81.49516210000002),
-                new google.maps.LatLng(12.9715987,77.5945627),
-                new google.maps.LatLng(53.41291,-8.24389),
-                new google.maps.LatLng(34.0583995,-106.8914159),
-                new google.maps.LatLng(-9.189967,-75.015152),
-                new google.maps.LatLng(55.6760968,12.5683371),
-                new google.maps.LatLng(53.9807737,-6.7148821),
-                new google.maps.LatLng(31.230416,121.473701),
-                new google.maps.LatLng(33.7489954,-84.3879824),
-                new google.maps.LatLng(8.4874949,76.948623),
-                new google.maps.LatLng(13.0524139,80.25082460000002),
-                new google.maps.LatLng(28.0836269,-80.60810889999999),
-                new google.maps.LatLng(39.0457549,-76.64127119999999),
-                new google.maps.LatLng(17.385044,78.486671),
-                new google.maps.LatLng(-23.5489433,-46.6388182),
-                new google.maps.LatLng(52.09179,5.114569899999999),
-                new google.maps.LatLng(19.2667,76.7833),
-                new google.maps.LatLng(19.0759837,72.8776559),
-                new google.maps.LatLng(12.9715987,77.5945627),
-                new google.maps.LatLng(41.1566892,-8.6239254),
-                new google.maps.LatLng(39.90403,116.407526),
-                new google.maps.LatLng(42.3584308,-71.0597732),
-                new google.maps.LatLng(32.725409,-97.3208496),
-                new google.maps.LatLng(39.074208,21.824312),
-                new google.maps.LatLng(10.066049,123.538599),
-                new google.maps.LatLng(37.42410599999999,-122.1660756),
-                new google.maps.LatLng(36.204824,138.252924)
-              ];
-
-        var map;
-        var markers = [];
-        var iterator = 0;
-        var dropped = false;
-        var recentered = false;
-
-        function initialize() {
-            var mapOptions = {
-              center: new google.maps.LatLng(0,0),
-              //draggable: false,
-              //zoomControl: false,
-              scrollwheel: false,
-              streetViewControl: false,
-              mapTypeControl: false,
-              panControl: false,
-              //disableDoubleClickZoom: true,
-              zoom: 2,
-              //disableDefaultUI: true,
-              mapTypeId: google.maps.MapTypeId.ROADMAP
-            };
-            map = new google.maps.Map(document.getElementById("map-canvas"),
-                mapOptions);
-
-            google.maps.event.addDomListener(map, 'idle', function() {
-                        if (recentered == false) {
-                            map.setCenter(new google.maps.LatLng(20,0));
-                            recentered = true;
-                        }
-                    });
-
-            var bounds = new google.maps.LatLngBounds();
-
-            // Extend bounds with each point
-            for (var i = 0; i < points.length; i++) {
-            bounds.extend(points[i]);
-            }
-            map.fitBounds(bounds);
-
-            if (document.hasFocus()) {
-                dropped = true;
-                drop();
-            }
-
-
-        }
-        function drop() {
-            dropped = true;
-            for (var i = 0; i < points.length; i++) {
-                setTimeout(function() {
-                    addMarker();
-                }, i * 50);
-            }
-        }
-
-        function addMarker() {
-            markers.push(new google.maps.Marker({
-                position: points[iterator],
-                map: map,
-                draggable: false,
-                animation: google.maps.Animation.DROP
-            }));
-            iterator++;
-        }
-
-        google.maps.event.addDomListener(window, 'load', initialize);
-
-        window.onfocus = function () {
-            if (dropped == false) {
-                drop();
-            }
-        };
-    </script>
-    
-
-    <!-- HTML5 shim, for IE6-8 support of HTML5 elements. All other JS at the end of file. -->
-    <!--[if lt IE 9]>
-      <script src="js/html5shiv.js"></script>
-      <script src="js/respond.min.js"></script>
-    <![endif]-->
-</head>
-<body>
-
-    <header class="navbar navbar-inverse navbar-fixed-top bf-docs-nav secondary" role="banner">
-    <div class="container">
-        <div class="navbar-header">
-            <button class="navbar-toggle" type="button" data-toggle="collapse" data-target=".bf-navbar-collapse">
-                <span class="sr-only">Toggle navigation</span>
-                <span class="icon-bar"></span>
-                <span class="icon-bar"></span>
-                <span class="icon-bar"></span>
-            </button>
-        </div>
-        <a href="/" class="navbar-brand">
-                <img id="home-logo" src="/img/apache_geode_logo_white_small.png" />
-            </a>
-        <nav class="collapse navbar-collapse bf-navbar-collapse" role="navigation">
-            <ul class="nav navbar-nav navbar-right">
-                <li class="active"><a href="/community/"><span class="icns icon-group"></span></a></li>
-                <li><a href="http://geode.docs.pivotal.io" target="_blank"><span class="icns icon-book"></span></a></li>
-                <li><a href="http://github.com/apache/incubator-geode" target="_blank"><span class="icns icon-github-sign"></span></a></li>
-                <!--<li><a href="https://trello.com/b/exQmJIOn/usergrid" target="_blank"><span class="icns icon-trello"></span></a></li>-->
-                <li><a href="https://issues.apache.org/jira/browse/GEODE/"
-                       target="_blank"><span class="icns icon-bug"></span></a></li>
-                <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank"><span class="icns icon-stackexchange"></span></a></li>
-                <li><a href="/community/#mailing-lists"><span class="icns icon-envelope"></span></a></li>
-                <!-- <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li> -->
-                <li><a href="https://twitter.com/apachegeode" target="_blank"><span class="icns icon-twitter"></span></a></li>
-                <li><a href="https://cwiki.apache.org/confluence/display/geode/" target="_blank"><span class="icns icon-edit"></span></a></li>
-                <li><a href="/releases/"><span class="icns icon-releases"></span></a></li>
-            </ul>
-        </nav>
-    </div>
-    </header>
-
-
-
-
-
-<!-- <div id="map-canvas" style="width: 100%;"></div> -->
-
-<section class="bf-tagline">
-    <div class="container">
-    	<div class="row">
-        <br/>
-	    	<h2>Join Our Community of Contributors!</h2>
-        <p>The Apache Geode (incubating) team welcomes contributors who want to support the Geode technology. Our community builds everything from this website, from the Geode code to documentation and best practices information.</p>
-
-        <p>We especially welcome additions and corrections to the documentation, wiki, and website to improve the user experience. Bug reports and fixes and additions to the Apache Geode code are welcome. Helping users learn best practices also earns good karma in our community.</p>
-		</div>
-	</div>
-</section>
-
-<section class="bf-community">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12">
-    	    	<h2 class="icns-envelope" id="mailing-lists"><span>Mailing-lists</span></h2>
-			</div>
-		</div>
-		<div class="row">
-	    	<div class="col-md-4">
-	    		<h3>Users</h3>
-	    		<p><em>Perfect if you build apps against Apache Geode or deploy Apache Geode.</em></p>
-	    		<p>To subscribe, send a blank email to<br/><a href="mailto:user-subscribe@geode.incubator.apache.org">user-subscribe@geode.incubator.apache.org</a>.</p>
-	    		<p>You can also <a href="http://markmail.org/search/?q=list%3Aorg.apache.geode.user+order%3Adate-backward">read the archives</a>.</p>
-			</div>
-	    	<div class="col-md-4">
-	    		<h3>Dev</h3>
-	    		<p><em>If you are building contributions & modifications to Apache Geode this is the list for you.</em><p>
-	    		<p>To subscribe, send a blank email to<br/><a href="mailto:dev-subscribe@geode.incubator.apache.org">dev-subscribe@geode.incubator.apache.org</a>.</p>
-	    		<p>You can also <a href="http://markmail.org/search/?q=list%3Aorg.apache.geode.dev+order%3Adate-backward">read the archives</a>.</p>
-			</div>
-	    	<div class="col-md-4">
-	    		<h3>Commits</h3>
-	    		<p><em>This list receives an email whenever new code is contributed to Apache Geode.</em><p>
-	    		<p>To subscribe, send a blank email to<br/><a href="mailto:commits-subscribe@geode.incubator.apache.org">commits-subscribe@geode.incubator.apache.org</a>.</p>
-	    		<p>You can also <a href="http://markmail.org/search/?q=list%3Aorg.apache.geode.commits+order%3Adate-backward">read the archives</a>.</p>
-			</div>
-		</div>
-	</div>
-</section>
-
-<section class="bf-community">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12" id="events">
-    	    	<h2 class="icns-calendar"><span>Events</span></h2>
-			</div>
-		</div>
-		<div class="row">
-    	<div class="col-md-3 done">
-    	    	<h3><a target="_blank" href="http://events.linuxfoundation.org/events/archive/2015/apachecon-north-america">ApacheCon </a> <small>Austin, TX <br/> April 13-16, 2015</small></h3>
-    	    	<p>
-              <ul>
-                <li>
-                  <a target="_blank" href="http://events.linuxfoundation.org/sites/events/files/slides/GemFire_ApacheCon.pdf">"Unleashing the Silicon Forest Fire - the Open Sourcing of GemFire"</a>
-                </li>
-                <li>
-                  <a target="_blank" href="http://events.linuxfoundation.org/sites/events/files/slides/apachecon15_markito_melo.pdf">"Implementing a Highly-Scalable Stock Prediction System with R, Geode and Spring XD"</a>
-                </li>
-              </ul>
-            </p>
-        </div>
-			<div class="col-md-3 done">
-    	    	<h3><a target="_blank" href="http://conferences.oreilly.com/oscon/open-source-2015">OSCON </a> <small>Portland, OR<br /> July 20-24, 2015</small></h3>
-            <p>
-              <ul>
-                <li><a target="_blank" href="http://conferences.oreilly.com/oscon/open-source-2015/public/schedule/detail/44875">Build your first Internet of Things App today with Open Source Software</a></li>
-              <ul/>
-            <p>
-			</div>
-      <div class="col-md-3 done">
-            <h3><a target="_blank" href="http://www.springone2gx.com/">SpringOne2GX</a> <small>Washington, DC<br /> September 14-17, 2015</small></h3>
-            <p>
-              <ul>
-                <li><a target="_blank" href="http://www.slideshare.net/SpringCentral/building-highly-scalable-spring-applications-using-inmemory-data-grids-53086251">Building highly-scalable Spring applications with in-memory, distributed data grids</a></li>
-
-                <li><a target="_blank" href="http://www.slideshare.net/SpringCentral/implementing-a-highly-scalable-stock-prediction-system-with-r-apache-geode-and-spring-xd">Implementing a highly scalable stock prediction system with R, Geode and Spring XD</a></li>
-              <ul/>
-            <p>
-      </div>
-    </div>
-    <div class="row">
-      <div class="col-md-3 done">
-        <h3><a target="_blank" href="http://events.linuxfoundation.org/events/apache-big-data-europe">Apache: Big Data </a> <small>Budapest, Hungary <br/> September 28-30, 2015</small></h3>
-        <p>
-        <ul>
-          <li><a target="_blank" href="http://events.linuxfoundation.org/sites/events/files/slides/ApacheConBigData%20-%20Introducing%20Apache%20Geode%20-%20final.pdf">An Introduction to Apache Geode (incubating)</a></li>
-          <li><a target="_blank" href="https://events.linuxfoundation.org/sites/events/files/slides/ApacheCon%20Big%20Data%202015%20-%20Implementing%20a%20Highly%20Scalable%20In-Memory%20Stock%20Prediction%20System%20with%20Apache%20Geode%20(incubating),%20R,%20SparkML%20and%20Spring%20XD.pdf">Implementing a Highly Scalable In-Memory Stock Prediction System with Apache Geode (incubating), R and Spring XD</a></li>
-        <ul/>
-        <p>
-      </div>
-      <div class="col-md-3 done">
-    	    	<h3><a target="_blank" href="http://www.slideshare.net/ApacheGeode/open-sourcing-gemfire-apache-geode">PJUG Meetup </a> <small>Portland, OR <br /> July 20-24, 2015</small></h3>
-            <p>Joint meeting with co-hosted between OSCON, PJUG and PDXScala<p>
-			</div>
-
-			<div class="col-md-3">
-				<h3>&nbsp;</h3>
-    	    	<p><i>Want to organize a Geode event? <a href="mailto:gregchase@apache.org">Contact us!</a></i><p>
-			</div>
-		</div>
-	</div>
-</section>
-
-<section class="bf-community">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12" id="live">
-    	    	<h2 class="icns-chat"><span>Live Chat and Geode Clubhouse</span></h2>
-			</div>
-		</div>
-		<div class="row">
-    	<div class="col-md-3">
-    	    	<h3><a target="_blank" href="http://s.apache.org/geodechat">HipChat</a></h3>
-    	    	<p>Some of the Geode team hangs around this HipChat Room: <a href="http://s.apache.org/geodechat" target="_blank">http://s.apache.org/geodechat</a><p>
-		  </div>
-      <div class="col-md-3">
-        	<h3><a target="_blank" href="http://stackoverflow.com/search?q=Apache%20Geode">StackOverflow</a></h3>
-        	<p>The Geode team is always ready to answer questions on <a href="http://stackoverflow.com/search?q=Apache%20Geode">StackOverflow</a><p>
-      </div>
-      <div class="col-md-3">
-        	<h3><a target="_blank" href="https://www.youtube.com/channel/UCaY2q0UlWjAgEGL7uhCLs6A">Geode ClubHouse</a></h3>
-        	<p>We meet every 15 days online for discussions around specific features, detailing internals and discuss on-going issues on JIRA at the Geode Clubhouse. All meetings are recorded and videos are availabe in our <a href="https://www.youtube.com/channel/UCaY2q0UlWjAgEGL7uhCLs6A">YouTube</a> channel.<p>
-      </div>
-	  </div>
-</section>
-
-<section class="bf-community">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12">
-    	    	<h2 class="icns-briefcase" id="deployments"><span>Deployments</span></h2>
-			</div>
-		</div>
-		<div class="row">
-			<a href="#">
-	    	    <div class="col-md-3">
-	    	    	<h3>Your project!</h3>
-	    	    	<p><p>
-				</div>
-			</a>
-			<!-- <a href="#">
-				<div class="col-md-3">
-	    	    	<h3>Project ABC</h3>
-	    	    	<p>Lorem ipsum dolor sit amet, consectetur adipiscing elit. Aliquam finibus cursus est. Morbi consequat velit ac aliquet elementum. Vivamus gravida blandit accumsan. Mauris commodo consectetur velit, quis malesuada velit semper ac. Vivamus egestas massa et placerat rhoncus.<p>
-				</div>
-			</a> -->
-			<div class="col-md-3">
-				<h3>&nbsp;</h3>
-    	    	<p><i>Want to be added to this list? <a href="mailto:user@geode.incubator.apache.org">Contact us!</a></i><p>
-			</div>
-		</div>
-	</div>
-</section>
-
-<section class="bf-community committers">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12">
-    	    	<h2 class="icns-group" id="committers"><span>Committers</span></h2>
-			</div>
-		</div>
-		<div class="row">
-      <br/>
-			<div class="col-md-2">
-        <li>Amey Barve</li>
-        <li>Adib Saikali</li>
-        <li>Alan Strait</li>
-        <li>Amogh Shetkar</li>
-        <li>Anil Gingade</li>
-        <li>Anilkumar Gingade</li>
-        <li>Anthony Baker</li>
-        <li>Ashvin Agrawal</li>
-        <li>Asif Shahid</li>
-        <li>Avinash Dongre</li>
-        <li>Barry Oglesby</li>
-        <li>Ben Reser</li>
-        <li>Bruce Schuchardt</li>
-        <li>Bruce Szalwinski</li>
-        <li>Catherine Johnson</li>
-        <li>Chip Childers</li>
-        <li>Christian Tzolov</li>
-        <li>Dan Smith</li>
-        <li>Darrel Schneider</li>
-        <li>Dave Muirhead</li>
-        <li>David Yozie</li>
-        <li>Dick Cavender</li>
-			</div>
-			<div class="col-md-2">
-        <li>Edin Zulich</li>
-        <li>Eric Shu</li>
-        <li>Gideon Low</li>
-        <li>Greg Chase</li>
-        <li>Hemant Bhanawat</li>
-        <li>Henry Saputra</li>
-        <li>Hitesh Khamesra</li>
-        <li>Jacob Barrett</li>
-        <li>Jags Ramnarayan</li>
-        <li>Jan Iversen</li>
-        <li>Jason Huynh</li>
-        <li>Jens Deppe</li>
-        <li>Jianxia Chen</li>
-        <li>John Blum</li>
-        <li>Justin Erenkrantz</li>
-        <li>Ketan Deshpande</li>
-        <li>Kirk Lund</li>
-        <li>Kishor Bachhav</li>
-        <li>Konstantin Boudnik</li>
-        <li>Konstantin Ignatyev</li>
-        <li>Lise Storc</li>
-			</div>
-			<div class="col-md-2">
-        <li>Luke Shannon</li>
-        <li>Lyndon Adams</li>
-        <li>Lynn Gallinat</li>
-        <li>Lynn Hughes-Godfrey</li>
-        <li>Mark Bretl</li>
-        <li>Michael Schubert</li>
-        <li>Namrata Thanvi</li>
-        <li>Neeraj Kumar</li>
-        <li>Nilkanth Patel</li>
-        <li>Qihong Chen</li>
-        <li>Rahul Diyewar</li>
-        <li>Randy May</li>
-        <li>Roman Shaposhnik</li>
-        <li>Severine Tymon</li>
-        <li>Shatarupa Nandi</li>
-        <li>Shirish Deshmukh</li>
-        <li>Sonal Agarwal</li>
-        <li>Soubhik Chakraborty</li>
-        <li>Sourabh Bansod</li>
-        <li>Stephane Maldini</li>
-			</div>
-			<div class="col-md-2">
-        <li>Stuart Williams</li>
-        <li>Sudhir Menon</li>
-        <li>Sunil Jigyasu</li>
-        <li>Supriya Pillai</li>
-        <li>Suranjan Kumar</li>
-        <li>Suyog Bhokare</li>
-        <li>Swapnil Bawaskar</li>
-        <li>Swati Sawant</li>
-        <li>Tushar Khairnar</li>
-        <li>Udo Kohlmeyer</li>
-        <li>Vince Ford</li>
-        <li>Vinesh Prasanna Manoharan</li>
-        <li>Vivek Bhaskar</li>
-        <li>Wes Williams</li>
-        <li>William A. Rowe Jr.</li>
-        <li>William Markito</li>
-        <li>Will Schipp</li>
-        <li>Xiaojian Zhou</li>
-        <li>Yogesh Mahajan</li>
-			</div>
-      <div class="col-md-3">
-        <h3>&nbsp;</h3>
-            <p><i>Want to be added to this list? Join the community, contribute and <a href="mailto:dev@geode.incubator.apache.org">contact us!</a></i><p>
-      </div>
-		</div>
-	</div>
-</section>
-
-<!-- <section class="bf-community tools">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12">
-    	    	<h2 class="icns-tools" id="tools"><span>Tools</span></h2>
-			</div>
-		</div>
-		 <div class="row">
-			<div class="col-md-2 text-center">
-				<a href="http://www.jetbrains.com/idea/" target="_blank"><img src="/img/intellij.png" /></a>
-				<p>IntelliJ IDEA</p>
-			</div>
-			<div class="col-md-2 text-center">
-				<a href="http://yourkit.com/" target="_blank"><img src="/img/yourkit.jpeg" /></a>
-				<p>YourKit Java Profiler</p>
-			</div>
-		</div>
-		These great companies have provided free product licenses to the Apache Geode team. We use these tools and love them.  Thank you!
-	</div>
-</section> -->
-
-
-
-
-<footer class="bf-footer" role="contentinfo">
-    <div class="container">
-        <div class="row">
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header"><a href="/">Home</a></li>
-                    <li class="nav-header"><a href="/community/">Community</a></li>
-                        <li><a href="/community/#events">Events</a></li>
-                        <li><a href="/community/#mailing-lists">Mailing Lists</a></li>
-                        <li><a href="/community/#deployments">Deployments</a></li>
-                        <!-- <li><a href="/community/#committers">Commiters</a></li> -->
-                </ul>
-            </div>
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header"><a href="http://github.com/apache/incubator-geode" target="_blank">Code</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Project+Proposals+and+Specifications" target="_blank">Specifications</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Geode+Internal+Architecture" target="_blank">Internal Architecture</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Writing+tests" target="_blank">Writing Tests</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Criteria+for+Code+Submissions" target="_blank">Code Submissions</a></li>
-                </ul>
-            </div>
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header">Resources</li>
-                    <li><a href="http://github.com/apache/geode-incubator" target="_blank">GitHub Code</a></li>
-                    <li><a href="http://geode.docs.pivotal.io" target="_blank">Docs</a></li>
-                    <li><a href="https://issues.apache.org/jira/browse/GEODE" target="_blank">JIRA Bug Tracker</a></li>
-                    <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank">StackOverflow</a></li>
-                    <li><a href="/community/#live">Live Chat</a></li>
-                    <li><a href="https://twitter.com/apachegeode" target="_blank">Twitter</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Index#Index-Geodein5minutesGeodein5minutes" target="_blank">Geode in 5 minutes</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/How+to+Contribute" target="_blank">How to Contribute</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Application+Development" target="_blank">Application Development</a></li>
-
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Technology+FAQ" target="_blank">FAQ</a></li>
-
-                </ul>
-            </div>
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header">Apache</li>
-                    <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                    <li><a href="http://www.apache.org/foundation/sponsorship.html" target="_blank">Sponsorship</a></li>
-                    <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                    <li><a href="http://www.apache.org/security/">Security</a></li>
-                    <li><a href="http://www.apache.org/" target="_blank">Apache Foundation</a></li>
-                </ul>
-            </div>
-            <div class="col-md-4">
-              <a class="twitter-timeline" href="https://twitter.com/search?q=%23ApacheGeode%20OR%20Apache%20Geode" data-widget-id="665030887004725248">Tweets about #ApacheGeode OR Apache Geode</a>
-<script>!function(d,s,id){var js,fjs=d.getElementsByTagName(s)[0],p=/^http:/.test(d.location)?'http':'https';if(!d.getElementById(id)){js=d.createElement(s);js.id=id;js.src=p+"://platform.twitter.com/widgets.js";fjs.parentNode.insertBefore(js,fjs);}}(document,"script","twitter-wjs");</script>
-            </div>
-        <!--
-            <div class="col-md-4">
-                <div class="input-group form-search">
-                    <input type="text" class="form-control search-query">
-                    <span class="input-group-btn">
-                        <button type="submit" class="btn btn-primary" data-type="last">Search</button>
-                    </span>
-                </div>
-            </div> -->
-        </div>
-        <div class="row">
-          <center>
-            <div id="copyright">
-                <a href="http://incubator.apache.org" target="_blank"><img src="/img/egg-logo.png" /></a><br/><br/>
-                <p class="credits">
-                Apache Geode is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
-                </p>
-                <p>Copyright © 2015 The Apache Software Foundation, Licensed under the Apache License, Version 2.0.<br>
-                Apache, Apache Geode (incubating), and the Apache feather logos are trademarks of The Apache Software Foundation.</p>
-                <p>Site designed & assembled with love by <a href="https://github.com/ryuneeee">@ryuneeee</a> + <a href="https://github.com/realbeast">@realbeast</a> + <a href="https://twitter.com/timanglade">@timanglade</a> + <a href="https://twitter.com/snoopdave">@snoopdave</a> for Apache Usergrid.</p>
-                <p>Modified for Apache Geode by <a href="https://twitter.com/william_markito">@william_markito</a>.</p>
-            </div>
-          </center>
-        </div>
-    </div>
-</footer>
-
-<script type="text/javascript" src="/js/head.js"></script>
-<script type="text/javascript">
-    head.js("/js/jquery-1.10.1.min.js", "/js/bootstrap.min.js", "/js/usergrid-site.js");
-</script>
-<!-- update the code below for google analytics -->
-<!--
- <script>
-  (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
-  (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
-  m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
-  })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
-
-  ga('create', 'UA-45815079-1', 'apache.org');
-  ga('send', 'pageview');
-
-</script> -->
-
-
-
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/css/bootflat-extensions.css
----------------------------------------------------------------------
diff --git a/gemfire-site/content/css/bootflat-extensions.css b/gemfire-site/content/css/bootflat-extensions.css
deleted file mode 100644
index 513ecaa..0000000
--- a/gemfire-site/content/css/bootflat-extensions.css
+++ /dev/null
@@ -1,356 +0,0 @@
-/*
-    Bootflat 1.0.1
-    Designed & Built by flathemes, http://www.flathemes.com
-    Licensed under MIT License, http://opensource.org/licenses/mit-license.html
-
-    Thanks for supporting our website and enjoy!
-*/
-/*------------------------------------*\
-    $bubble
-\*------------------------------------*/
-.bubble-body {
-    position: relative;
-    padding: 3px;
-    background-color: #ecf0f1;
-    border-radius: 3px;
-    overflow: visible;
-}
-.pull-left ~ .bubble-body {
-    margin-left: 60px;
-}
-.pull-right ~ .bubble-body {
-    margin-right: 60px;
-}
-.bubble-body .bubble-inner {
-    min-height: 32px;
-    border: 1px solid #d3d7d7;
-    background-color: #fbfbfb;
-}
-.bubble-body .bubble-heading {
-    padding: 0 10px;
-    border-bottom: 1px solid #d3d7d7;
-    background-color: #f4f4f4;
-    font-size: 12px;
-    font-weight: bold;
-    color: #222;
-    overflow: hidden;
-    white-space: nowrap;
-    text-overflow: ellipsis;
-    height: 33px;
-    line-height: 33px;
-}
-.bubble-body .bubble-content {
-    padding: 10px;
-    font-size: 13px;
-    overflow: auto;
-    width: 100%;
-    line-height: 1.7;
-}
-.bubble-body .bubble-inner {
-    margin-bottom: 0;
-    -webkit-border-radius: 0;
-    -moz-border-radius: 0;
-    border-radius: 0;
-}
-.bubble-body .bubble-inner .bubble {
-    margin: 0 10px;
-    padding-top: 10px;
-    border-top: 1px solid #ecf0f1;
-}
-.bubble-body .bubble-inner .bubble .bubble {
-    margin: 0;
-}
-.bubble-body .bubble-inner .bubble-body:before,
-.bubble-body .bubble-inner .bubble-body:after {
-    display: none;
-}
-.bubble-body .bubble-inner .bubble-body,
-.bubble-body .bubble-inner .bubble-inner {
-    padding: 0;
-    border: none;
-    background-color: transparent;
-}
-.bubble-body .bubble-inner .bubble-inner .bubble-heading {
-    padding: 0;
-    border-bottom: none;
-    background-color: transparent;
-    height: auto;
-    line-height: normal;
-}
-.bubble-body .bubble-inner .bubble-inner .bubble-content {
-    padding: 0;
-    font-size: 13px;
-    overflow: auto;
-    width: 100%;
-    line-height: 1.5;
-}
-.bubble-arrow-left:before,
-.bubble-arrow-right:after {
-    position: absolute;
-    top: 15px;
-    content: "";
-    display: block;
-    height: 0;
-    width: 0;
-    border-width: 10px;
-    border-style: solid;
-}
-.bubble-arrow-left:before {
-    border-color: transparent #ecf0f1 transparent transparent;
-    left: -20px;
-}
-.bubble-arrow-right:after {
-    border-color: transparent transparent transparent #ecf0f1;
-    right: -20px;
-}
-/*------------------------------------*\
-    $breadcrumb-arrow
-\*------------------------------------*/
-.breadcrumb-arrow {
-    padding: 0;
-    list-style:none;
-    background-color: #ecf0f1;
-    height:36px;
-    line-height: 36px;
-}
-.breadcrumb-arrow li:first-child a {
-    border-top-left-radius: 4px;
-    border-bottom-left-radius: 4px;
-}
-.breadcrumb-arrow li,
-.breadcrumb-arrow li a,
-.breadcrumb-arrow li span{
-    display:-moz-inline-box;
-    display:inline-table;
-    display:inline-block;
-    zoom:1;
-    *display:inline;
-    vertical-align:top;
-}
-.breadcrumb-arrow li:not(:first-child) {
-    margin-left: -5px;
-}
-.breadcrumb-arrow li + li:before {
-    padding: 0;
-    content: "";
-}
-.breadcrumb-arrow li span {
-    padding: 0 10px;
-}
-.breadcrumb-arrow li a,
-.breadcrumb-arrow li:not(:first-child) span {
-    padding:0 10px 0 25px;
-    height:35px;
-    line-height:35px;
-}
-.breadcrumb-arrow li:first-child a {
-    padding: 0 10px;
-}
-.breadcrumb-arrow li a {
-    position:relative;
-    border:1px solid #3da8e3;
-    color:#fff;
-    background-color:#3da8e3;
-    text-decoration:none;
-}
-.breadcrumb-arrow li [class^="icon-"], 
-.breadcrumb-arrow ul li [class*=" icon-"] {
-    top: 0;
-}
-.breadcrumb-arrow-arrow li:first-child a {
-    padding-left:10px;
-}
-.breadcrumb-arrow li a:before,
-.breadcrumb-arrow li a:after {
-    position:absolute;
-    top:0;
-    content:'';
-    width: 0;
-    height: 0;
-    border-top: 17px solid transparent;
-    border-bottom: 17px solid transparent;
-}
-.breadcrumb-arrow li a:before {
-    right: -10px;
-    border-left-width: 10px;
-    border-left-style:solid;
-    border-left-color:#3da8e3;
-    z-index:3;
-}
-.breadcrumb-arrow li a:after{
-    right: -11px;
-    border-left: 10px solid #2980b9;
-    z-index:2;
-}
-.breadcrumb-arrow li a:hover,
-.breadcrumb-arrow li a:focus {
-    background-color:#3598ce;
-    border: 1px solid #3598ce;
-}
-.breadcrumb-arrow li a:hover:before,
-.breadcrumb-arrow li a:focus:before {
-    border-left-color: #3598ce;
-}
-.breadcrumb-arrow li a:active {
-    background-color:#2980b9;
-    border: 1px solid #2980b9;
-}
-.breadcrumb-arrow li a:active:before,
-.breadcrumb-arrow li a:active:after {
-    border-left-color:#2980b9;
-}
-.breadcrumb-arrow li span{
-    color:#bdc3c7;
-}
-/*------------------------------------*\
-    $nav-tabs-panel
-\*------------------------------------*/
-.nav-tabs-panel,
-.nav-tabs-panel.nav-justified {
-    margin-bottom: 15px;
-    border-bottom: 1px solid #2986b9;
-    background-color: #ecf0f1;
-}
-.nav-tabs-panel .tab-default,
-.nav-tabs-panel.nav-justified .tab-default {
-    margin-right: 0;
-    padding: 11px 15px;
-    border-bottom: none;
-    color: #292929;
-}
-.nav-tabs-panel.nav-justified .active .tab-default,
-.nav-tabs-panel.nav-justified .active .tab-default:hover,
-.nav-tabs-panel.nav-justified .active .tab-default:focus,
-.nav-tabs-panel .active .tab-default,
-.nav-tabs-panel .active .tab-default:hover,
-.nav-tabs-panel .active .tab-default:focus {
-    border-color: transparent transparent #2986b9 transparent;
-    border-bottom-style:solid;
-    border-width: 0 0 3px 0;
-    color: #fff;
-    background-color: #3da8e3;
-    -webkit-border-radius: 0;
-       -moz-border-radius: 0;
-            border-radius: 0;
-}
-.nav-tabs-panel li a:hover,
-.nav-tabs-panel li a:focus {
-    border-color: transparent transparent transparent;
-    background-color: transparent;
-}
-.nav-tabs-panel .open .dropdown-toggle,
-.nav-tabs-panel li.dropdown.open.active a:hover,
-.nav-tabs-panel li.dropdown.open.active a:focus {
-    color: #292929;
-    background-color: transparent;
-    border-color: transparent;
-}
-.nav-tabs-panel .dropdown-toggle .caret,
-.nav-tabs-panel .dropdown-toggle:hover .caret,
-.nav-tabs-panel .dropdown-toggle:focus .caret,
-.nav-tabs-panel li.dropdown.open .caret,
-.nav-tabs-panel li.dropdown.open.active .caret,
-.nav-tabs-panel li.dropdown.open a:hover .caret,
-.nav-tabs-panel li.dropdown.open a:focus .caret {
-    border-top-color: #292929;
-    border-bottom-color: #292929;
-}
-.nav-tabs-panel .active .dropdown-toggle .caret {
-    border-top-color: #fff;
-    border-bottom-color: #fff;
-}
-.nav-tabs-panel .dropdown-menu {
-    margin-top: 1px;
-}
-.nav-tabs-panel .dropdown-menu li a {
-    background-color: transparent;
-}
-.nav-tabs-panel .dropdown-menu li.active a {
-    background-color: #2986b9;
-}
-.nav-tabs-panel .dropdown-menu li a:hover,
-.nav-tabs-panel .dropdown-menu li a:focus {
-    background-color: #2986b9;
-}
-/*------------------------------------*\
-    $tabs-below
-\*------------------------------------*/
-.tabs-below .nav-tabs-panel {
-    margin-top: 15px;
-    margin-bottom: 0;
-    border-top: 1px solid #2986b9;
-    border-bottom:none;
-}
-.tabs-below .nav-tabs-panel li {
-    margin-top: 0;
-}
-.tabs-below .nav-tabs-panel li a:hover,
-.tabs-below .nav-tabs-panel li a:focus {
-    border-top-color: transparent;
-}
-.tabs-below .nav-tabs-panel .active .tab-default,
-.tabs-below .nav-tabs-panel .active .tab-default:hover,
-.tabs-below .nav-tabs-panel .active .tab-default:focus {
-    border-bottom-color: #2986b9;
-}
-.tabs-below .nav-tabs-panel .dropdown-menu {
-    -webkit-border-radius: 4px 4px 0 0;
-    -moz-border-radius: 4px 4px 0 0;
-    border-radius: 4px 4px 0 0;
-}
-/*------------------------------------*\
-    $tabs-left and $tabs-right
-\*------------------------------------*/
-.tabs-left .nav-tabs-panel,
-.tabs-right .nav-tabs-panel {
-    position: relative;
-    border-bottom: none;
-    z-index: 20;
-}
-.tabs-left .nav-tabs-panel li,
-.tabs-right .nav-tabs-panel li {
-    float: none;
-}
-.tabs-left .nav-tabs-panel li .tab-default,
-.tabs-right .nav-tabs-panel li .tab-default {
-    min-width: 39px;
-    margin-bottom:0;
-}
-.tabs-left .nav-tabs-panel li .tab-default:hover,
-.tabs-left .nav-tabs-panel li .tab-default:focus,
-.tabs-right .nav-tabs-panel li .tab-default:hover,
-.tabs-right .nav-tabs-panel li .tab-default:focus {
-    border-color: transparent;
-}
-.tabs-left .nav-tabs-panel {
-    float: left;
-    margin-right: 15px;
-    border-right: 1px solid #2986b9;
-}
-.tabs-left .nav-tabs-panel li a {
-    margin-right: 0;
-}
-.tabs-left .nav-tabs-panel .active .tab-default,
-.tabs-left .nav-tabs-panel .active .tab-default:hover,
-.tabs-left .nav-tabs-panel .active .tab-default:focus {
-    border-color: transparent transparent transparent #2986b9;
-    border-style: solid;
-    border-width: 0 0 0 3px;
-}
-.tabs-right .nav-tabs-panel {
-    float: right;
-    margin-left: 15px;
-    border-left: 1px solid #2986b9;
-}
-.tabs-right .nav-tabs-panel li a {
-    margin-left: 0;
-}
-.tabs-right .nav-tabs-panel .active .tab-default,
-.tabs-right .nav-tabs-panel .active .tab-default:hover,
-.tabs-right .nav-tabs-panel .active .tab-default:focus {
-    border-color: transparent #2986b9 transparent transparent;
-    border-style: solid;
-    border-width: 0 3px 0 0;
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/css/bootflat-square.css
----------------------------------------------------------------------
diff --git a/gemfire-site/content/css/bootflat-square.css b/gemfire-site/content/css/bootflat-square.css
deleted file mode 100644
index 0e448ab..0000000
--- a/gemfire-site/content/css/bootflat-square.css
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
-    Bootflat 1.0.1
-    Designed & Built by flathemes, http://www.flathemes.com
-    Licensed under MIT License, http://opensource.org/licenses/mit-license.html
-
-    Thanks for supporting our website and enjoy!
-*/
-/*------------------------------------*\
-    $default-square
-\*------------------------------------*/
-.img-thumbnail-square,
-.btn-square,
-.btn-group-square .btn,
-.btn-group-square .dropdown-menu,
-.btn-group-square .btn,
-.btn-group-square .btn:first-child:not(:last-child),
-.btn-group-square .btn:last-child:not(:first-child),
-.table-bordered-square,
-.table-bordered-square tr:first-child th:first-child,
-.table-bordered-square tr:first-child th:last-child,
-.table-bordered-square tr:last-child td:first-child,
-.table-bordered-square tr:last-child td:last-child,
-.input-group-square .btn,
-.input-group-square .dropdown-menu,
-.input-group-square .form-control,
-.input-group-square .input-group-addon,
-.dropdown-menu-square,
-.dropdown-menu-square .dropdown-submenu .dropdown-menu,
-.form-square .form-control,
-.form-square .btn,
-.form-control-square,
-.label-square,
-.badge-square,
-.alert-square,
-.alert-square .btn,
-.progress-square,
-.breadcrumb-square,
-.tooltip-square .tooltip-inner,
-.popover-square,
-.nav-list-panel-square,
-.nav-tabs-square li > a,
-.tabs-below .nav-tabs-square li > a,
-.tabs-right .nav-tabs-square li > a,
-.tabs-left .nav-tabs-square li > a,
-.nav-tabs-square .dropdown-menu,
-.tabs-below .nav-tabs-square .dropdown-menu,
-.nav-pills-square li a,
-.nav-pills-square .dropdown-menu,
-.navbar-square,
-.navbar-square .dropdown-menu,
-.pagination-square li:first-child a, 
-.pagination-square li:first-child span,
-.pagination-square li:last-child a, 
-.pagination-square li:last-child span,
-.pager-square li a:hover,
-.pager-square li a:focus,
-.panel-group-square .panel,
-.panel-group-square .panel-heading,
-.panel-group-square .panel-body,
-/*------------------------------------*\
-    $extend-square
-\*------------------------------------*/
-.breadcrumb-arrow-square li:first-child a {
-    -webkit-border-radius: 0;
-    -moz-border-radius: 0;
-    border-radius: 0;
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/css/bootflat.css
----------------------------------------------------------------------
diff --git a/gemfire-site/content/css/bootflat.css b/gemfire-site/content/css/bootflat.css
deleted file mode 100644
index 0f2f45e..0000000
--- a/gemfire-site/content/css/bootflat.css
+++ /dev/null
@@ -1,1559 +0,0 @@
-/*
-    Bootflat 1.0.1
-    Designed & Built by flathemes, http://www.flathemes.com
-    Licensed under MIT License, http://opensource.org/licenses/mit-license.html
-
-    Thanks for supporting our website and enjoy!
-*/
-/*------------------------------------*\
-    $typography
-\*------------------------------------*/
-a {
-    text-decoration: none;
-}
-a:hover {
-    text-decoration: underline;
-}
-a {
-    color: #2986b9;
-}
-a:hover {
-    color: #1b71a0;
-}
-ins {
-    background-color: #ff0;
-    text-decoration: none;
-}
-dfn[title] {
-    cursor: help;
-    border-bottom: 1px dotted;
-    font-style: normal;
-}
-q, blockquote {
-    font-style: italic;
-}
-q::before {
-    content: open-quote;
-}
-q::after {
-    content: close-quote;
-}
-hr.hr-line {
-  display: block;
-  padding: 0;
-  border: 0;
-  border-top: 1px solid #d3d7d7;
-}
-.first-letter::first-letter {
-    font-size: 5em;
-    line-height: 0.8em;
-    float: left;
-    position: relative;
-    padding-right: 6px;
-    font-weight: normal;
-}
-/*------------------------------------*\
-    $close
-\*------------------------------------*/
-.close:hover, .close:focus {
-    outline: none;
-}
-/*------------------------------------*\
-    $text
-\*------------------------------------*/
-.muted {
-    color: #d3d7d7;
-}
-a.muted:hover,
-a.muted:focus {
-    color: #aeb1b1;
-}
-.text-warning {
-    color: #d7af0d;
-}
-a.text-warning:hover,
-a.text-warning:focus {
-    color: #b1900b;
-}
-.text-danger,
-.text-error {
-    color: #c0392b;
-}
-a.text-danger:hover,
-a.text-danger:focus,
-a.text-error:hover,
-a.text-error:focus {
-    color: #a5281b;
-}
-.text-info {
-    color: #4fbeba;
-}
-a.text-info:hover,
-a.text-info:focus {
-    color: #1ba5a0;
-}
-.text-success {
-    color: #64b92a;
-}
-a.text-success:hover,
-a.text-success:focus {
-    color: #50a118;
-}
-.text-inverse {
-    color: #34495e;
-}
-a.text-inverse:hover,
-a.text-inverse:focus {
-    color: #263544;
-}
-/*------------------------------------*\
-    $lead
-\*------------------------------------*/
-.lead {
-    margin: 20px 0;
-    padding: 15px 30px 15px 15px;
-    border-left: 5px solid #d3d7d7;
-    font-size: 14px;
-    background-color: #f9f9f9;
-}
-/*------------------------------------*\
-    $page
-\*------------------------------------*/
-.page-header {
-    margin: 0 0 20px;
-}
-.page-header h1 {
-    margin-top: 0;
-}
-.page-article:before,
-.page-article:after {
-    display: table;
-    content: " ";
-}
-.page-article:after {
-    clear: both;
-}
-.page-article .meta {
-    font-size: 12px;
-    line-height: 18px;
-    color: #aeb1b1;
-}
-/*------------------------------------*\
-    $thumbnail
-\*------------------------------------*/
-a.thumbnail:hover, a.thumbnail:focus {
-    border-color: #2986b9;
-}
-.thumbnail .caption {
-    color: #292929;
-}
-.thumbnail .caption h1,
-.thumbnail .caption h2,
-.thumbnail .caption h3 {
-    margin-top: 9px;
-    font-size: 16px;
-    font-weight: bold;
-}
-/*------------------------------------*\
-    $list-group
-\*------------------------------------*/
-.list-group-item:hover,
-.list-group-item:active {
-    color: inherit;
-}
-.list-group-item.active,
-.list-group-item.active:hover,
-.list-group-item.active:focus {
-    background-color: #2986b9;
-    border-color: #2986b9;
-}
-/*------------------------------------*\
-    $btn
-\*------------------------------------*/
-.btn:focus {
-    outline: none;
-}
-.btn,
-.btn.disabled,
-.btn[disabled] {
-    border-color: #d3d7d7;
-    background-color: #d3d7d7;
-}
-.btn.disabled,
-.btn[disabled] {
-    opacity: .45;
-    filter: alpha(opacity=45);
-}
-.btn {
-    -webkit-transition: border-color 0.3s ease-out, background-color 0.3s ease-out;
-    -moz-transition: border-color 0.3s ease-out, background-color 0.3s ease-out;
-    transition: border-color 0.3s ease-out, background-color 0.3s ease-out;
-    color: #333;
-}
-.btn:hover,
-.btn:focus,
-.btn:active,
-.btn.active {
-    border-color: #aeb1b1;
-    background-color: #aeb1b1;
-}
-.btn-link {
-    border: 1px solid transparent !important;
-    color: #2986b9;
-    background-color: transparent;
-    -webkit-border-radius: 4px;
-    -moz-border-radius: 4px;
-    border-radius: 4px;
-}
-.btn-link.btn-lg {
-    -webkit-border-radius: 6px;
-    -moz-border-radius: 6px;
-    border-radius: 6px;
-}
-.btn-link.btn-sm,
-.btn-link.btn-xs {
-    -webkit-border-radius: 3px;
-    -moz-border-radius: 3px;
-    border-radius: 3px;
-}
-.btn-link:hover,
-.btn-link:focus {
-    text-decoration: none;
-}
-.btn-link:hover,
-.btn-link:focus,
-.btn-link:active,
-.btn-link.active {
-    color: #333;
-    border-color: #d3d7d7;
-    background-color: #d3d7d7;
-}
-.btn-link.disabled,
-.btn-link[disabled] {
-    border-color: #d3d7d7;
-    background-color: #d3d7d7;
-    color: #333;
-}
-.btn-default {
-    margin-left: 0;
-    /*border: 1px solid #ddd !important;*/
-    background-color: #fff;
-}
-.btn-default:hover,
-.btn-default:focus,
-.btn-default:active,
-.btn-default.active,
-.open .dropdown-toggle.btn-default {
-    border-color: #aeb1b1;
-    background-color: #d3d7d7;
-}
-.btn-default.disabled,
-.btn-default[disabled] {
-    background-color: #fff;
-}
-.btn-primary,
-.btn-info,
-.btn-success,
-.btn-warning,
-.btn-danger,
-.btn-inverse {
-    color: #fff;
-}
-.btn-primary,
-.btn-primary.disabled,
-.btn-primary[disabled] {
-    border-color: #2986b9;
-    background-color: #2986b9;
-}
-.btn-primary:hover,
-.btn-primary:focus,
-.btn-primary:active,
-.btn-primary.active,
-.open .dropdown-toggle.btn-primary {
-    border-color: #1b71a0;
-    background-color: #1b71a0;
-}
-.btn-info,
-.btn-info.disabled,
-.btn-info[disabled] {
-    border-color: #4fbeba;
-    background-color: #4fbeba;
-}
-.btn-info:hover,
-.btn-info:focus,
-.btn-info:active,
-.btn-info.active,
-.open .dropdown-toggle.btn-info {
-    border-color: #1ba5a0;
-    background-color: #1ba5a0;
-}
-.btn-success,
-.btn-success.disabled,
-.btn-success[disabled] {
-    border-color: #64b92a;
-    background-color: #64b92a;
-}
-.btn-success:hover,
-.btn-success:focus,
-.btn-success:active,
-.btn-success.active,
-.open .dropdown-toggle.btn-success {
-    border-color: #50a118;
-    background-color: #50a118;
-}
-.btn-warning,
-.btn-warning.disabled,
-.btn-warning[disabled] {
-    border-color: #d7af0d;
-    background-color: #d7af0d;
-}
-.btn-warning:hover,
-.btn-warning:focus,
-.btn-warning:active,
-.btn-warning.active,
-.open .dropdown-toggle.btn-warning {
-    border-color: #b1900b;
-    background-color: #b1900b;
-}
-.btn-danger,
-.btn-danger.disabled,
-.btn-danger[disabled] {
-    border-color: #c0392b;
-    background-color: #c0392b;
-}
-.btn-danger:hover,
-.btn-danger:focus,
-.btn-danger:active,
-.btn-danger.active,
-.open .dropdown-toggle.btn-danger {
-    border-color: #a5281b;
-    background-color: #a5281b;
-}
-.btn-inverse,
-.btn-inverse.disabled,
-.btn-inverse[disabled] {
-    border-color: #3c3c3c;
-    background-color: #1e90ff;
-    color: #fff;
-}
-.btn-inverse:hover,
-.btn-inverse:focus,
-.btn-inverse:active,
-.btn-inverse.active
-.open .dropdown-toggle.btn-inverse {
-    border-color: #3c3c3c;
-    background-color: #7b48fe;
-    color: #fff;
-}
-.btn-inverse .caret {
-    border-top-color: #fff;
-}
-.dropup .btn-inverse .caret {
-    border-bottom-color: #fff;
-}
-/*------------------------------------*\
-    $btn-group
-\*------------------------------------*/
-.btn-group .btn {
-    border-left-color: #aeb1b1;
-    border-right-color: #aeb1b1;
-}
-.btn-group .btn:first-child {
-    border-left-color: #d3d7d7;
-}
-.btn-group .btn:last-child,
-.btn-group .btn + .dropdown-toggle {
-    border-right-color: #d3d7d7;
-}
-.btn-group-vertical .btn:first-child {
-    border-top-color: #d3d7d7;
-}
-.btn-group-vertical .btn:last-child {
-    border-bottom-color: #d3d7d7;
-}
-.btn-group-vertical .btn,
-.btn-group-vertical .btn-group .btn {
-    border-top-color: #aeb1b1;
-    border-bottom-color: #aeb1b1;
-}
-.btn-group .btn-default {
-    border-left-color: #d3d7d7;
-    border-right-color: #d3d7d7;
-}
-.btn-group .btn-default:hover,
-.btn-group .btn-default:focus,
-.btn-group .btn-default:active,
-.btn-group .btn-default.active,
-.btn-group-vertical .btn-default:hover,
-.btn-group-vertical .btn-default:focus,
-.btn-group-vertical .btn-default:active,
-.btn-group-vertical .btn-default.active {
-    border-top-color: #d3d7d7;
-    border-bottom-color: #d3d7d7;
-}
-.btn-group-vertical .btn-default,
-.btn-group-vertical .btn-group .btn-default,
-.btn-group-vertical .btn-default {
-    border-top-color: #d3d7d7 !important;
-    border-bottom-color: #d3d7d7;
-}
-.btn-group .btn-primary {
-    border-left-color: #1b71a0;
-    border-right-color: #1b71a0;
-}
-.btn-group .btn-primary:first-child {
-    border-left-color: #2986b9;
-}
-.btn-group .btn-primary:last-child,
-.btn-group .btn-primary + .dropdown-toggle {
-    border-right-color: #2986b9;
-}
-.btn-group-vertical .btn-primary:first-child {
-    border-top-color: #1b71a0;
-}
-.btn-group-vertical .btn-primary,
-.btn-group-vertical .btn-group .btn-primary,
-.btn-group-vertical .btn-primary:last-child {
-    border-bottom-color: #2986b9;
-}
-.btn-group-vertical .btn-primary {
-    border-top-color: #1b71a0;
-    border-bottom-color: #1b71a0;
-}
-.btn-group .btn-info {
-    border-left-color: #1ba5a0;
-    border-right-color: #1ba5a0;
-}
-.btn-group .btn-info:first-child {
-    border-left-color: #4fbeba;
-}
-.btn-group .btn-info:last-child,
-.btn-group .btn-info + .dropdown-toggle {
-    border-right-color: #4fbeba;
-}
-.btn-group-vertical .btn-info:first-child {
-    border-top-color: #1ba5a0;
-}
-.btn-group-vertical .btn-info,
-.btn-group-vertical .btn-group .btn-info,
-.btn-group-vertical .btn-info:last-child {
-    border-bottom-color: #4fbeba;
-}
-.btn-group-vertical .btn-info {
-    border-top-color: #1ba5a0;
-    border-bottom-color: #1ba5a0;
-}
-.btn-group .btn-success {
-    border-left-color: #50a118;
-    border-right-color: #50a118;
-}
-.btn-group .btn-success:first-child {
-    border-left-color: #64b92a;
-}
-.btn-group .btn-success:last-child,
-.btn-group .btn-success + .dropdown-toggle {
-    border-right-color: #64b92a;
-}
-.btn-group-vertical .btn-success:first-child {
-    border-top-color: #50a118;
-}
-.btn-group-vertical .btn-success,
-.btn-group-vertical .btn-group .btn-success,
-.btn-group-vertical .btn-success:last-child {
-    border-bottom-color: #64b92a;
-}
-.btn-group-vertical .btn-success {
-    border-top-color: #50a118;
-    border-bottom-color: #50a118;
-}
-.btn-group .btn-warning {
-    border-left-color: #b1900b;
-    border-right-color: #b1900b;
-}
-.btn-group .btn-warning:first-child {
-    border-left-color: #d7af0d;
-}
-.btn-group .btn-warning:last-child,
-.btn-group .btn-warning + .dropdown-toggle {
-    border-right-color: #d7af0d;
-}
-.btn-group-vertical .btn-warning:first-child {
-    border-top-color: #b1900b;
-}
-.btn-group-vertical .btn-warning,
-.btn-group-vertical .btn-group .btn-warning,
-.btn-group-vertical .btn-warning:last-child {
-    border-bottom-color: #d7af0d;
-}
-.btn-group-vertical .btn-warning {
-    border-top-color: #b1900b;
-    border-bottom-color: #b1900b;
-}
-.btn-group .btn-danger {
-    border-left-color: #a5281b;
-    border-right-color: #a5281b;
-}
-.btn-group .btn-danger:first-child {
-    border-left-color: #c0392b;
-}
-.btn-group .btn-danger:last-child,
-.btn-group .btn-danger + .dropdown-toggle {
-    border-right-color: #c0392b;
-}
-.btn-group-vertical .btn-danger:first-child {
-    border-top-color: #a5281b;
-}
-.btn-group-vertical .btn-danger,
-.btn-group-vertical .btn-group .btn-danger,
-.btn-group-vertical .btn-danger:last-child {
-    border-bottom-color: #c0392b;
-}
-.btn-group-vertical .btn-danger {
-    border-top-color: #a5281b;
-    border-bottom-color: #a5281b;
-}
-.btn-group .btn-inverse {
-    border-left-color: #2c3e50;
-    border-right-color: #2c3e50;
-}
-.btn-group .btn-inverse:first-child {
-    border-left-color: #34495e;
-}
-.btn-group .btn-inverse:last-child,
-.btn-group .btn-inverse + .dropdown-toggle {
-    border-right-color: #34495e;
-}
-.btn-group-vertical .btn-inverse:first-child {
-    border-top-color: #2c3e50;
-}
-.btn-group-vertical .btn-inverse,
-.btn-group-vertical .btn-group .btn-inverse,
-.btn-group-vertical .btn-inverse:last-child {
-    border-bottom-color: #34495e;
-}
-.btn-group-vertical .btn-inverse {
-    border-top-color: #2c3e50;
-    border-bottom-color: #2c3e50;
-}
-/*------------------------------------*\
-    $btn-group-justified
-\*------------------------------------*/
-.btn-group-justified .btn {
-    border-right: none;
-}
-/*------------------------------------*\
-    $input-group-btn
-\*------------------------------------*/
-.input-group-btn .btn + .btn {
-    border-left: 1px solid #aeb1b1;
-}
-.input-group-btn .btn + .btn.btn-default {
-    margin-left: -5px;
-    border-left: 1px solid #d3d7d7;
-}
-.input-group-btn .btn + .btn.btn-primary {
-    border-left: 1px solid #1b71a0;
-}
-.input-group-btn .btn + .btn.btn-info {
-    border-left: 1px solid #1ba5a0;
-}
-.input-group-btn .btn + .btn.btn-success {
-    border-left: 1px solid #50a118;
-}
-.input-group-btn .btn + .btn.btn-warning {
-    border-left: 1px solid #b1900b;
-}
-.input-group-btn .btn + .btn.btn-danger {
-    border-left: 1px solid #a5281b;
-}
-.input-group-btn .btn + .btn.btn-inverse {
-    border-left: 1px solid #263544;
-}
-/*------------------------------------*\
-    $dropdown-menu
-\*------------------------------------*/
-.dropdown-menu {
-    border: none;
-    -webkit-border-radius: 4px;
-    -moz-border-radius: 4px;
-    border-radius: 4px;
-    background-color: #292929;
-}
-.dropdown-menu .dropdown-header {
-    font-size: 14px;
-    font-weight: bold;
-    padding: 5px 20px;
-}
-.dropdown-menu li a {
-    padding:5px 20px;
-    color: #fff;
-}
-.dropdown-menu li a:hover,
-.dropdown-menu li a:focus,
-.dropdown-menu .active a,
-.dropdown-menu .active a:hover,
-.dropdown-menu .active a:focus{
-    background-color: #2986b9;
-    -webkit-transition: 0.25s;
-       -moz-transition: 0.25s;
-            transition: 0.25s;
-    -webkit-backface-visibility: hidden;
-    outline: none;
-}
-.dropdown-menu .disabled a,
-.dropdown-menu .disabled a:hover,
-.dropdown-menu .disabled a:focus {
-    color:#9B9B9B !important;
-    cursor: default;
-}
-.dropdown-menu .divider {
-    margin:3px 0 0;
-    background-color: #373737;
-    border-bottom: none;
-}
-/*------------------------------------*\
-    $dropdown-submenu
-\*------------------------------------*/
-.dropdown-submenu {
-    position: relative;
-}
-.dropdown-submenu .dropdown-menu {
-    top: 0;
-    left: 100%;
-    margin-top: -6px;
-    margin-left: -1px;
-    -webkit-border-radius: 0 4px 4px 4px;
-    -moz-border-radius: 0 4px 4px 4px;
-    border-radius: 0 4px 4px 4px;
-}
-.dropdown-submenu:hover .dropdown-menu {
-    display: block;
-}
-.dropup .dropdown-submenu .dropdown-menu {
-    top: auto;
-    bottom: 0;
-    margin-top: 0;
-    margin-bottom: -2px;
-    -webkit-border-radius: 4px 4px 4px 0;
-    -moz-border-radius: 4px 4px 4px 0;
-    border-radius: 4px 4px 4px 0;
-}
-.dropdown-submenu > a:after {
-    display: block;
-    content: " ";
-    float: right;
-    width: 0;
-    height: 0;
-    border-color: transparent;
-    border-style: solid;
-    border-width: 5px 0 5px 5px;
-    border-left-color: #fff;
-    margin-top: 5px;
-    margin-right: -10px;
-}
-.dropdown-submenu:hover a:after {
-    border-left-color: #ffffff;
-}
-.dropdown-submenu.pull-left {
-    float: none;
-}
-.dropdown-submenu.pull-left .dropdown-menu {
-    left: -100%;
-    margin-left: 10px;
-    -webkit-border-radius: 4px 0 4px 4px;
-    -moz-border-radius: 4px 0 4px 4px;
-    border-radius: 4px 0 4px 4px;
-}
-/*------------------------------------*\
-    $table
-\*------------------------------------*/
-.table {
-    background-color: #fff;
-    border-collapse: separate;
-    -webkit-border-radius: 4px;
-    -moz-border-radius: 4px;
-    border-radius: 4px;
-}
-.table .success td,
-.table .danger td,
-.table .warning td,
-.table .info td,
-.table .active td {
-    color: #fff;
-}
-.table .success th,
-.table .success td {
-    border-color: #64b92a !important;
-    background-color: #64b92a !important;
-}
-.table .danger th,
-.table .danger td {
-    border-color: #c0392b !important;
-    background-color: #c0392b !important;
-}
-.table .warning th,
-.table .warning td {
-    border-color: #d7af0d !important;
-    background-color: #d7af0d !important;
-}
-.table .info th,
-.table .info td {
-    border-color: #4fbeba !important;
-    background-color: #4fbeba !important;
-}
-.table .active th,
-.table .active td {
-    border-color: #2986b9 !important;
-    background-color: #2986b9 !important;
-}
-.table-background thead {
-    color:#fff;
-    background-color:#d3d7d7;
-}
-.table-background thead tr th,
-.table-background thead tr td {
-    border-bottom: none;
-}
-.table-bordered tr:first-child th:first-child {
-    -webkit-border-top-left-radius: 4px;
-    -moz-border-radius-topleft: 4px;
-    border-top-left-radius: 4px;
-}
-.table-bordered tr:first-child th:last-child {
-    -webkit-border-top-right-radius: 4px;
-    -moz-border-radius-topright: 4px;
-    border-top-right-radius: 4px;
-}
-.table-bordered tr:last-child td:first-child {
-    -webkit-border-bottom-left-radius: 4px;
-    -moz-border-radius-bottomleft: 4px;
-    border-bottom-left-radius: 4px;
-}
-.table-bordered tr:last-child td:last-child {
-    -webkit-border-bottom-right-radius: 4px;
-    -moz-border-radius-bottomright: 4px;
-    border-bottom-right-radius: 4px;
-}
-.table-bordered {
-    border-width: 1px 1px 0 0;
-}
-.table-bordered thead tr th,
-.table-bordered tbody tr th,
-.table-bordered tfoot tr th,
-.table-bordered thead tr td,
-.table-bordered tbody tr td,
-.table-bordered tfoot tr td {
-    border-width: 0 0 1px 1px;
-}
-.table-hover tbody tr td,
-.table-hover tbody tr th {
-    -webkit-transition: all 0.3s ease-out;
-    -moz-transition: all 0.3s ease-out;
-    transition: all 0.3s ease-out;
-}
-.table-hover tbody tr:hover td,
-.table-hover tbody tr:hover th,
-.table-striped tbody tr:nth-child(odd) td,
-.table-striped tbody tr:nth-child(odd) th {
-    background-color: #ecf0f1;
-}
-.table-hover tbody tr:hover td,
-.table-hover tbody tr:hover th {
-    background-color: #f9f9f9;
-}
-.table-hover .success:hover td,
-.table-hover .success:hover th {
-    border-color: #79d738 !important;
-    background-color: #79d738 !important;
-}
-.table-hover .danger:hover td,
-.table-hover .danger:hover th {
-    border-color: #d44637 !important;
-    background-color: #d44637 !important;
-}
-.table-hover .warning:hover td,
-.table-hover .warning:hover th {
-    border-color: #f1c40f !important;
-    background-color: #f1c40f !important;
-}
-.table-hover .info:hover td,
-.table-hover .info:hover th {
-    border-color: #4cd1cb !important;
-    background-color: #4cd1cb !important;
-}
-.table-hover .active:hover td,
-.table-hover .active:hover th {
-    border-color: #3598ce !important;
-    background-color: #3598ce !important;
-}
-/*------------------------------------*\
-    $form
-\*------------------------------------*/
-.form-control {
-    padding: 4px 6px;
-    border-width: 1px;
-    border-style: solid;
-    border-color: #ddd #eee #eee #ddd;
-    -webkit-box-shadow: none;
-    -moz-box-shadow: none;
-    box-shadow: none;
-}
-.form-control:-moz-placeholder,
-.form-control::-moz-placeholder,
-.form-control:-ms-input-placeholder,
-.form-control::-webkit-input-placeholder,
-.form-control.placeholder {
-    color: #bdc3c7;
-}
-.form-control:focus {
-    -webkit-transition: 0.25s;
-       -moz-transition: 0.25s;
-            transition: 0.25s;
-    -webkit-backface-visibility: hidden;
-    outline:none;
-    -webkit-box-shadow: none;
-    -moz-box-shadow: none;
-    box-shadow: none;
-}
-.form-control[disabled],
-.form-control[readonly],
-fieldset[disabled] .form-control {
-    cursor: not-allowed;
-    opacity: 0.4;
-    filter: alpha(opacity=4);
-}
-
-.form-inline .form-group,
-.form-inline .checkbox,
-.form-inline .radio,
-.form-inline .btn {
-    margin-right: 5px;
-}
-/*------------------------------------*\
-    $validation states
-\*------------------------------------*/
-.has-warning .form-control,
-.has-warning .form-control:focus,
-.has-error .form-control,
-.has-error .form-control:focus,
-.has-info .form-control,
-.has-info .form-control:focus,
-.has-success .form-control,
-.has-success .form-control:focus {
-    -webkit-box-shadow: none;
-    -moz-box-shadow: none;
-    box-shadow: none;
-    border-radius: auto;
-}
-.has-warning .help-block,
-.has-warning .control-label,
-.has-warning .form-control {
-    color: #d7af0d;
-}
-.has-warning .form-control {
-    border-color:#d7af0d;
-}
-.has-error .help-block,
-.has-error .control-label,
-.has-error .form-control {
-    color: #c0392b;
-}
-.has-error .form-control {
-    border-color: #c0392b;
-}
-.has-info .help-block,
-.has-info .control-label,
-.has-info .form-control {
-    color: #4fbeba;
-}
-.has-info .form-control {
-    border-color: #4fbeba;
-}
-.has-success .help-block,
-.has-success .control-label,
-.has-success .form-control {
-    color: #64b92a;
-}
-.has-success .form-control {
-    border-color:#64b92a;
-}
-/*------------------------------------*\
-    $input-group-addon
-\*------------------------------------*/
-.input-group-addon {
-    background-color: #d3d7d7;
-    border: 1px solid #d3d7d7;
-}
-.input-group-addon .radio,
-.input-group-addon .checkbox {
-    margin: -2px 0 -4px !important;
-}
-/*------------------------------------*\
-    $search-query
-\*------------------------------------*/
-.form-search .search-query,
-.form-search .search-query:first-child,
-.form-search .search-query:last-child {
-    padding: 0 17px;
-    -webkit-border-radius: 17px;
-    -moz-border-radius: 17px;
-    border-radius: 17px;
-}
-.input-group .form-control:last-child {
-    padding: 0 17px 0 10px;
-    border-bottom-left-radius: 0;
-    border-top-left-radius: 0;
-}
-.input-group .form-control:first-child {
-    border-bottom-right-radius: 0;
-    border-top-right-radius: 0;
-}
-.form-search .btn {
-    -webkit-border-radius: 25px;
-    -moz-border-radius: 25px;
-    border-radius: 25px;
-}
-.search-only {
-    position: relative;
-}
-.search-only:before {
-    position: absolute;
-    top: 1px;
-    left: 8px;
-    width: 30px;
-    line-height: 30px;
-    text-align: center;
-    font-family: "FontAwesome";
-    font-size: 18px;
-    color: #d3d7d7;
-    content: "\f002";
-    z-index: 20;
-}
-.search-only .form-control:last-child {
-    padding-left: 40px;
-}
-/*------------------------------------*\
-    $radio and $checkbox
-\*------------------------------------*/
-.radio, .checkbox {
-    padding-left:0;
-    margin-top: 0;
-}
-.checkbox label,
-.radio label{
-    display:inline-block;
-    vertical-align:top;
-    height:24px;
-    line-height:24px;
-    font-weight: normal;
-    cursor:pointer;
-}
-.checkbox .icheckbox_flat,
-.radio .iradio_flat{
-    background-image: url(../img/check_flat/default.png);
-}
-.checkbox .icheckbox_flat,
-.radio .iradio_flat{
-    display:inline-block;
-    vertical-align:top;
-    margin: 0;
-    padding: 0;
-    width: 24px;
-    height: 24px;
-    border: none;
-    cursor: pointer;
-    background-repeat:no-repeat;
-}
-.checkbox .icheckbox_flat {
-    background-position: 0 0;
-}
-.checkbox .icheckbox_flat.hover {
-    background-position: -24px 0;
-}
-.checkbox .icheckbox_flat.checked {
-    background-position: -48px 0;
-}
-fieldset[disabled] .checkbox .icheckbox_flat,
-.checkbox .icheckbox_flat.disabled {
-    background-position: -72px 0;
-    cursor: not-allowed;
-}
-.checkbox .icheckbox_flat.checked.disabled {
-    background-position: -96px 0;
-}
-.radio .iradio_flat {
-    background-position: -120px 0;
-}
-.radio .iradio_flat.hover {
-    background-position: -144px 0;
-}
-.radio .iradio_flat.checked {
-    background-position: -168px 0;
-}
-fieldset[disabled] .radio .iradio_flat,
-.radio .iradio_flat.disabled {
-    background-position: -192px 0;
-    cursor: not-allowed;
-}
-.radio .iradio_flat.checked.disabled {
-    background-position: -216px 0;
-}
-fieldset[disabled] .checkbox,
-fieldset[disabled] .radio,
-.checkbox .disabled,
-.checkbox .checked.disabled,
-.radio .disabled,
-.radio .checked.disabled {
-    color:#bdc3c7;
-    cursor: not-allowed;
-}
-.radio-inline {
-    margin-left: 10px;
-}
-/*------------------------------------*\
-    $label and $badge
-\*------------------------------------*/
-.label, .badge {
-    background-color: #d3d7d7;
-}
-.label.label-primary,
-.badge.badge-primary,
-.label.label-info,
-.badge.badge-info,
-.label.label-success,
-.badge.badge-success,
-.label.label-warning,
-.badge.badge-warning,
-.label.label-danger,
-.badge.badge-danger,
-.label.label-inverse,
-.badge.badge-inverse {
-    color: #fff;
-}
-.label.label-primary,
-.badge.badge-primary {
-    background-color: #2986b9;
-}
-.label.label-info,
-.badge.badge-info {
-    background-color: #4fbeba;
-}
-.label.label-success,
-.badge.badge-success {
-    background-color: #64b92a;
-}
-.label.label-warning,
-.badge.badge-warning {
-    background-color: #d7af0d;
-}
-.label.label-danger,
-.badge.badge-danger {
-    background-color: #c0392b;
-}
-.label.label-inverse,
-.badge.badge-inverse {
-    background-color: #34495e;
-}
-/*------------------------------------*\
-    $alert
-\*------------------------------------*/
-.alert {
-    background-color: #fece10;
-    border: 1px solid #fece10;
-}
-.alert, .alert h4{
-    color: #FFF;
-}
-.alert .alert-link,
-.alert .alert-link:hover,
-.alert .alert-link:focus {
-    color: #000;
-    opacity: 0.55;
-    filter: alpha(opacity=55);
-}
-.alert h4{
-    margin-bottom: 10px;
-    font-weight: bold;
-}
-.alert-dismissable .close {
-    color: #000;
-}
-.alert.alert-info {
-    background-color: #5eddd8;
-    border: 1px solid #5eddd8;
-}
-.alert.alert-danger,
-.alert.alert-error {
-    background-color: #e74c3c;
-    border: 1px solid #e74c3c;
-}
-.alert.alert-success {
-    background-color: #87eb41;
-    border: 1px solid #87eb41;
-}
-/*------------------------------------*\
-    $popover
-\*------------------------------------*/
-.popover {
-    background-color: #292929;
-    color:#FFF;
-    border: 1px solid #292929;
-}
-.popover-title {
-    padding-bottom: 0;
-    font-weight: bold;
-    background-color: transparent;
-    border-bottom: none;
-}
-.popover .close {
-    position: absolute;
-    top:10px;
-    right: 10px;
-}
-.popover.top .arrow,
-.popover.top .arrow:after {
-    border-top-color: #292929;
-}
-.popover.right .arrow,
-.popover.right .arrow:after {
-    border-right-color: #292929;
-}
-.popover.bottom .arrow,
-.popover.bottom .arrow:after {
-    border-bottom-color: #292929;
-}
-.popover.left .arrow,
-.popover.left .arrow:after {
-    border-left-color: #292929;
-}
-/*------------------------------------*\
-    $pagination
-\*------------------------------------*/
-.pagination .active a,
-.pagination .active span,
-.pagination .active a:hover,
-.pagination .active span:hover,
-.pagination .active a:focus,
-.pagination .active span:focus {
-    background-color: #2986b9;
-    border-color: #2986b9;
-}
-/*------------------------------------*\
-    $pager
-\*------------------------------------*/
-.pager li a, .pager li span {
-    border: none;
-    -webkit-border-radius: 0;
-    -moz-border-radius: 0;
-    border-radius: 0;
-}
-.pager li a:hover,
-.pager li a:focus {
-    color: #fff;
-    background-color: #2986b9;
-    -webkit-border-radius: 4px;
-       -moz-border-radius: 4px;
-            border-radius: 4px;
-}
-/*------------------------------------*\
-    $progress
-\*------------------------------------*/
-.progress,
-.progress .progress-bar {
-    -webkit-box-shadow: none !important;
-    -moz-box-shadow: none !important;
-    box-shadow: none !important;
-}
-.progress {
-    height: 12px;
-    overflow: hidden;
-    background-color: #ecf0f1;
-}
-.progress .progress-bar {
-    background-color: #2986b9;
-}
-.progress-success .progress-bar,
-.progress .progress-bar-success,
-.progress-success.progress-striped .progress-bar,
-.progress-striped .progress-bar-success {
-    background-color: #64b92a;
-}
-.progress-info .progress-bar,
-.progress .progress-bar-info,
-.progress-info.progress-striped .progress-bar,
-.progress-striped .progress-bar-info {
-    background-color: #4fbeba;
-}
-.progress-danger .progress-bar,
-.progress .progress-bar-danger,
-.progress-danger.progress-striped .progress-bar,
-.progress-striped .progress-bar-danger {
-    background-color: #c0392b;
-}
-.progress-warning .progress-bar,
-.progress .progress-bar-warning,
-.progress-warning.progress-striped .progress-bar,
-.progress-striped .progress-bar-warning {
-    background-color: #d7af0d;
-}
-/*------------------------------------*\
-    $breadcrumb
-\*------------------------------------*/
-.breadcrumb {
-    background-color: #ecf0f1;
-}
-/*------------------------------------*\
-    $nav
-\*------------------------------------*/
-.nav .open > a,
-.nav .open > a:hover,
-.nav .open > a:focus {
-    border-color: transparent;
-}
-/*------------------------------------*\
-    $navbar-toggle
-\*------------------------------------*/
-.navbar-toggle:focus {
-    outline: none;
-}
-/*------------------------------------*\
-    $navbar
-\*------------------------------------*/
-.navbar .divider-vertical {
-    border-left-width: 1px;
-    border-left-style: solid;
-    height: 50px;
-}
-.navbar-default {
-    border:none;
-    background-color:#3da8e3;
-}
-.navbar-default .navbar-brand {
-    color: #fff;
-}
-.navbar-default .navbar-link:hover,
-.navbar-default .navbar-link:focus,
-.navbar-default .navbar-brand:hover,
-.navbar-default .navbar-brand:focus {
-    color: #d3efff;
-}
-.navbar-default .navbar-link,
-.navbar-default .navbar-text {
-    color: #d3efff;
-}
-.navbar-default .dropdown-header {
-    color: #aeb1b1;
-}
-.navbar-default .divider-vertical {
-    border-left-color: #2986b9;
-}
-.navbar-default .nav li a,
-.navbar-default .nav li a:focus,
-.navbar-default .nav li a:hover {
-    color: #fff;
-}
-.navbar-default .nav .active a,
-.navbar-default .nav .active a:hover,
-.navbar-default .nav .active a:focus {
-    color: #fff;
-    background-color: #2986b9;
-}
-.navbar-default .nav li.dropdown.open .dropdown-toggle,
-.navbar-default .nav li.dropdown.active .dropdown-toggle,
-.navbar-default .nav li.dropdown.open.active .dropdown-toggle {
-    background-color: #2986b9;
-    color: #fff;
-}
-.navbar-default .nav li.dropdown a:hover .caret,
-.navbar-default .nav li.dropdown a:focus .caret,
-.navbar-default .nav li.dropdown .dropdown-toggle .caret,
-.navbar-default .nav li.dropdown.open .dropdown-toggle .caret,
-.navbar-default .nav li.dropdown.active .dropdown-toggle .caret,
-.navbar-default .nav li.dropdown.open.active .dropdown-toggle .caret {
-    border-top-color: #fff;
-    border-bottom-color: #fff;
-}
-.navbar-default .nav li .dropdown-menu:before,
-.navbar-default .nav li .dropdown-menu:after {
-    border-bottom-color: transparent;
-}
-.navbar-default .navbar-toggle {
-    background-color: #aeb1b1;
-}
-.navbar-default .navbar-toggle:hover,
-.navbar-default .navbar-toggle:focus {
-    background-color: #c2c2c2;
-}
-.navbar-default .navbar-collapse,
-.navbar-default .navbar-form {
-    border-color: #aeb1b1;
-}
-/*------------------------------------*\
-    $navbar-inverse
-\*------------------------------------*/
-.navbar-inverse {
-    background-color:#292929;
-}
-.navbar-inverse .nav .active a,
-.navbar-inverse .nav .active a:hover,
-.navbar-inverse .nav .active a:focus,
-.navbar-inverse .nav li.dropdown.open .dropdown-toggle,
-.navbar-inverse .nav li.dropdown.active .dropdown-toggle,
-.navbar-inverse .nav li.dropdown.open.active .dropdown-toggle{
-    background-color: #000;
-}
-.navbar-inverse .divider-vertical {
-    border-left-color: #000;
-}
-.navbar-inverse .navbar-form .form-control {
-    border: 1px solid #292929;
-}
-.navbar-inverse .navbar-form .form-control:focus,
-.navbar-inverse .navbar-form .form-control.focused {
-    color: #292929;
-    background-color: #fff !important;
-    border: 1px solid #000;
-}
-/*------------------------------------*\
-    $navbar-right
-\*------------------------------------*/
-.navbar-right .dropdown.open .dropdown-toggle {
-    -webkit-border-radius: 0 4px 0 0;
-    -moz-border-radius: 0 4px 0 0;
-    border-radius: 0 4px 0 0;
-}
-/*------------------------------------*\
-    $nav-list
-\*------------------------------------*/
-.nav-list {
-    padding: 10px 0;
-}
-.nav-list li a,
-.nav-list .nav-header {
-    padding: 5px 15px;
-}
-.nav-list .nav-header {
-    font-weight: bold;
-}
-.nav-list li a {
-    color: #292929;
-}
-.nav-list li a:hover,
-.nav-list li a:focus,
-.nav-list .active a,
-.nav-list .active a:hover,
-.nav-list .active a:focus {
-    color: #3498db;
-    text-decoration: none;
-}
-.nav-list li a:hover,
-.nav-list li a:focus {
-    background-color: transparent;
-}
-.nav-list .active a,
-.nav-list .active a:hover,
-.nav-list .active a:focus {
-    font-weight: bold;
-}
-.nav-list .divider {
-    margin: 9px 15px;
-    overflow: hidden;
-    border-bottom: 1px solid #ddd;
-}
-.nav-list  .nav-list-sub {
-    list-style: none;
-}
-.nav-list  .nav-list-sub {
-    padding-left: 0;
-}
-.nav-list  .nav-list-sub li a {
-    padding: 2px 15px 2px 30px;
-    display: block;
-}
-
-.nav-list-panel {
-    -webkit-border-radius: 5px;
-    -moz-border-radius: 5px;
-    border-radius: 5px;
-    background-color:#ecf0f1;
-}
-.nav-list-panel li a:hover,
-.nav-list-panel li a:focus,
-.nav-list-panel .active a,
-.nav-list-panel .active a:hover,
-.nav-list-panel .active a:focus {
-    color: #fff;
-    background-color: #2986b9;
-    text-decoration: none;
-}
-.nav-list-panel li a:hover,
-.nav-list-panel li a:focus {
-    opacity: 0.45;
-    filter: alpha(opacity=45);
-}
-.nav-list-panel .active a,
-.nav-list-panel .active a:hover,
-.nav-list-panel .active a:focus {
-    opacity: 1;
-    filter: alpha(opacity=100);
-}
-/*------------------------------------*\
-    $nav-tabs
-\*------------------------------------*/
-.nav-tabs > li > a {
-    color: #292929;
-}
-.nav-tabs .dropdown-toggle .caret,
-.nav-tabs .dropdown-toggle:hover .caret,
-.nav-tabs .dropdown-toggle:focus .caret,
-.nav-tabs li.dropdown.open .caret,
-.nav-tabs li.dropdown.open.active .caret,
-.nav-tabs li.dropdown.open a:hover .caret,
-.nav-tabs li.dropdown.open a:focus .caret,
-.nav-tabs .active .dropdown-toggle .caret {
-    border-top-color: #292929;
-    border-bottom-color: #292929;
-}
-.tabs-below .nav-tabs {
-    border-top: 1px solid #ddd;
-    border-bottom: none;
-}
-.tabs-below .nav-tabs .dropdown-menu {
-    -webkit-border-radius: 4px 4px 0 0;
-    -moz-border-radius: 4px 4px 0 0;
-    border-radius: 4px 4px 0 0;
-}
-.tabs-below .nav-tabs li {
-    margin-top: -1px;
-    margin-bottom: 0;
-}
-.tabs-below .nav-tabs li a {
-    -webkit-border-radius: 0 0 4px 4px;
-       -moz-border-radius: 0 0 4px 4px;
-            border-radius: 0 0 4px 4px;
-}
-.tabs-below .nav-tabs li a:hover,
-.tabs-below .nav-tabs li a:focus {
-    border-top-color: #ddd;
-    border-bottom-color: transparent;
-}
-.tabs-below .nav-tabs .active a,
-.tabs-below .nav-tabs .active a:hover,
-.tabs-below .nav-tabs .active a:focus {
-    border-color: transparent #ddd #ddd #ddd;
-}
-.tabs-left .nav-tabs,
-.tabs-right .nav-tabs {
-    border-bottom: none;
-}
-.tabs-left .nav-tabs li,
-.tabs-right .nav-tabs li {
-    float: none;
-}
-.tabs-left .nav-tabs li a,
-.tabs-right .nav-tabs li a {
-    min-width: 74px;
-    margin-right: 0;
-    margin-bottom: 3px;
-}
-.tabs-left .nav-tabs {
-    float: left;
-    margin-right: 19px;
-    border-right: 1px solid #ddd;
-}
-.tabs-left .nav-tabs li > a {
-    margin-right: -1px;
-    -webkit-border-radius: 4px 0 0 4px;
-       -moz-border-radius: 4px 0 0 4px;
-            border-radius: 4px 0 0 4px;
-}
-.tabs-left .nav-tabs li a:hover,
-.tabs-left .nav-tabs li a:focus {
-    border-color: #eeeeee #dddddd #eeeeee #eeeeee;
-}
-.tabs-left .nav-tabs .active a,
-.tabs-left .nav-tabs .active a:hover,
-.tabs-left .nav-tabs .active a:focus {
-    border-color: #ddd transparent #ddd #ddd;
-    *border-right-color: #ffffff;
-}
-.tabs-right .nav-tabs {
-    float: right;
-    margin-left: 19px;
-    border-left: 1px solid #ddd;
-}
-.tabs-right .nav-tabs li a {
-    margin-left: -1px;
-    -webkit-border-radius: 0 4px 4px 0;
-       -moz-border-radius: 0 4px 4px 0;
-            border-radius: 0 4px 4px 0;
-}
-.tabs-right .nav-tabs li a:hover,
-.tabs-right .nav-tabs li a:focus {
-    border-color: #eeeeee #eeeeee #eeeeee #dddddd;
-}
-.tabs-right .nav-tabs .active a,
-.tabs-right .nav-tabs .active a:hover,
-.tabs-right .nav-tabs .active a:focus {
-    border-color: #ddd #ddd #ddd transparent;
-    *border-left-color: #ffffff;
-}
-/*------------------------------------*\
-    $nav-pills
-\*------------------------------------*/
-.nav-pills > li > a {
-    color: #292929;
-}
-.nav-pills > li > a:hover,
-.nav-pills > li > a:focus {
-    background-color: ;
-}
-.nav-pills > li.active > a,
-.nav-pills > li.active > a:hover,
-.nav-pills > li.active > a:focus {
-    background-color: #2986b9;
-    color:#fff;
-}
-.nav-pills .dropdown-toggle .caret,
-.nav-pills .open .dropdown-toggle:focus .caret,
-.nav-pills .dropdown-toggle:hover .caret,
-.nav-pills .dropdown-toggle:focus .caret {
-    border-top-color: #292929;
-    border-bottom-color: #292929;
-}
-.nav-pills .open .dropdown-toggle,
-.nav-pills .open .dropdown-toggle:focus {
-    background-color: transparent;
-}
-/*------------------------------------*\
-    $breadcrumb
-\*------------------------------------*/
-.breadcrumb > li + li::before {
-    font-family: 'FontAwesome';
-    content: "\f105";
-}
-/*------------------------------------*\
-    $panel-group
-\*------------------------------------*/
-.panel-group .panel {
-    -webkit-box-shadow: none;
-    -moz-box-shadow: none;
-    box-shadow: none;
-}
-.panel-group .panel-heading {
-    padding: 9px 15px;
-    background-color: #2986b9;
-}
-.panel-group .panel-heading a,
-.panel-group .panel-heading a:hover,
-.panel-group .panel-heading a:focus,
-.panel-group .panel-heading a:active {
-    color:#fff;
-    text-decoration: none;
-}
-.panel-group .panel-body {
-    border: 1px solid #ddd;
-    -webkit-border-radius: 0 0 4px 4px;
-    -moz-border-radius: 0 0 4px 4px;
-    border-radius: 0 0 4px 4px;
-}


[17/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/js/jquery.icheck.js
----------------------------------------------------------------------
diff --git a/gemfire-site/content/js/jquery.icheck.js b/gemfire-site/content/js/jquery.icheck.js
deleted file mode 100644
index c92faa0..0000000
--- a/gemfire-site/content/js/jquery.icheck.js
+++ /dev/null
@@ -1,397 +0,0 @@
-/*!
- * iCheck v0.8, http://git.io/uhUPMA
- * =================================
- * Powerful jQuery plugin for checkboxes and radio buttons customization
- *
- * (c) 2013 Damir Foy, http://damirfoy.com
- * MIT Licensed
- */
-
-(function($, _iCheck, _checkbox, _radio, _checked, _disabled, _type, _click, _touch, _add, _remove, _cursor) {
-
-  // Create a plugin
-  $.fn[_iCheck] = function(options, fire) {
-
-    // Cached vars
-    var user = navigator.userAgent,
-      ios = /ipad|iphone|ipod/i.test(user),
-      handle = ':' + _checkbox + ', :' + _radio;
-
-    // Check if we should operate with some method
-    if (/^(check|uncheck|toggle|disable|enable|update|destroy)$/.test(options)) {
-
-      // Find checkboxes and radio buttons
-      return this.each(function() {
-        var self = $(this),
-          tree = self.is(handle) ? self : self.find(handle);
-
-        tree.each(function() {
-          self = $(this);
-
-          if (options == 'destroy') {
-            tidy(self, 'ifDestroyed');
-          } else {
-            operate(self, true, options);
-          };
-
-          // Fire method's callback
-          if ($.isFunction(fire)) {
-            fire();
-          };
-        });
-      });
-
-    // Customization
-    } else if (typeof options == 'object' || !options) {
-
-      //  Check if any options were passed
-      var settings = $.extend({
-          checkedClass: _checked,
-          disabledClass: _disabled,
-          labelHover: true
-        }, options),
-
-        selector = settings.handle,
-        hoverClass = settings.hoverClass || 'hover',
-        focusClass = settings.focusClass || 'focus',
-        activeClass = settings.activeClass || 'active',
-        labelHover = !!settings.labelHover,
-        labelHoverClass = settings.labelHoverClass || 'hover',
-
-        // Setup clickable area
-        area = ('' + settings.increaseArea).replace('%', '') | 0;
-
-      // Selector limit
-      if (selector == _checkbox || selector == _radio) {
-        handle = ':' + selector;
-      };
-
-      // Clickable area limit
-      if (area < -50) {
-        area = -50;
-      };
-
-      // Walk around the selector
-      return this.each(function() {
-        var self = $(this),
-          tree = self.is(handle) ? self : self.find(handle);
-
-        tree.each(function() {
-          self = $(this);
-
-          // If already customized
-          tidy(self);
-
-          var node = this,
-            id = node.id,
-
-            // Layer styles
-            offset = -area + '%',
-            size = 100 + (area * 2) + '%',
-            layer = {
-              position: 'absolute',
-              top: offset,
-              left: offset,
-              display: 'block',
-              width: size,
-              height: size,
-              margin: 0,
-              padding: 0,
-              background: '#fff',
-              border: 0,
-              opacity: 0
-            },
-
-            // Choose how to hide input
-            hide = ios || /android|blackberry|windows phone|opera mini/i.test(user) ? {
-              position: 'absolute',
-              visibility: 'hidden'
-            } : area ? layer : {
-              position: 'absolute',
-              opacity: 0
-            },
-
-            // Get proper class
-            className = node[_type] == _checkbox ? settings.checkboxClass || 'i' + _checkbox : settings.radioClass || 'i' + _radio,
-
-            // Find assigned labels
-            label = $('label[for="' + id + '"]').add(self.closest('label')),
-
-            // Wrap input
-            parent = self.wrap('<div class="' + className + '"/>').trigger('ifCreated').parent().append(settings.insert),
-
-            // Layer addition
-            helper = $('<ins class="' + _iCheck + '-helper"/>').css(layer).appendTo(parent);
-
-          // Finalize customization
-          self.data(_iCheck, {o: settings, s: self.attr('style')}).css(hide);
-          !!settings.inheritClass && parent[_add](node.className);
-          !!settings.inheritID && id && parent.attr('id', _iCheck + '-' + id);
-          parent.css('position') == 'static' && parent.css('position', 'relative');
-          operate(self, true, 'update');
-
-          // Label events
-          if (label.length) {
-            label.on(_click + '.i mouseenter.i mouseleave.i ' + _touch, function(event) {
-              var type = event[_type],
-                item = $(this);
-
-              // Do nothing if input is disabled
-              if (!node[_disabled]) {
-
-                // Click
-                if (type == _click) {
-                  operate(self, false, true);
-
-                // Hover state
-                } else if (labelHover) {
-                  if (/ve|nd/.test(type)) {
-                    // mouseleave|touchend
-                    parent[_remove](hoverClass);
-                    item[_remove](labelHoverClass);
-                  } else {
-                    parent[_add](hoverClass);
-                    item[_add](labelHoverClass);
-                  };
-                };
-
-                if (ios) {
-                  event.stopPropagation();
-                } else {
-                  return false;
-                };
-              };
-            });
-          };
-
-          // Input events
-          self.on(_click + '.i focus.i blur.i keyup.i keydown.i keypress.i', function(event) {
-            var type = event[_type],
-              key = event.keyCode;
-
-            // Click
-            if (type == _click) {
-              return false;
-
-            // Keydown
-            } else if (type == 'keydown' && key == 32) {
-              if (!(node[_type] == _radio && node[_checked])) {
-                if (node[_checked]) {
-                  off(self, _checked);
-                } else {
-                  on(self, _checked);
-                };
-              };
-
-              return false;
-
-            // Keyup
-            } else if (type == 'keyup' && node[_type] == _radio) {
-              !node[_checked] && on(self, _checked);
-
-            // Focus/blur
-            } else if (/us|ur/.test(type)) {
-              parent[type == 'blur' ? _remove : _add](focusClass);
-            };
-          });
-
-          // Helper events
-          helper.on(_click + ' mousedown mouseup mouseover mouseout ' + _touch, function(event) {
-            var type = event[_type],
-
-              // mousedown|mouseup
-              toggle = /wn|up/.test(type) ? activeClass : hoverClass;
-
-            // Do nothing if input is disabled
-            if (!node[_disabled]) {
-
-              // Click
-              if (type == _click) {
-                operate(self, false, true);
-
-              // Active and hover states
-              } else {
-
-                // State is on
-                if (/wn|er|in/.test(type)) {
-                  // mousedown|mouseover|touchbegin
-                  parent[_add](toggle);
-
-                // State is off
-                } else {
-                  parent[_remove](toggle + ' ' + activeClass);
-                };
-
-                // Label hover
-                if (label.length && labelHover && toggle == hoverClass) {
-
-                  // mouseout|touchend
-                  label[/ut|nd/.test(type) ? _remove : _add](labelHoverClass);
-                };
-              };
-
-              if (ios) {
-                event.stopPropagation();
-              } else {
-                return false;
-              };
-            };
-          });
-        });
-      });
-    } else {
-      return this;
-    };
-  };
-
-  // Do something with inputs
-  function operate(input, direct, method) {
-    var node = input[0];
-
-      // disable|enable
-      state = /ble/.test(method) ? _disabled : _checked,
-      active = method == 'update' ? {checked: node[_checked], disabled: node[_disabled]} : node[state];
-
-    // Check and disable
-    if (/^ch|di/.test(method) && !active) {
-      on(input, state);
-
-    // Uncheck and enable
-    } else if (/^un|en/.test(method) && active) {
-      off(input, state);
-
-    // Update
-    } else if (method == 'update') {
-
-      // Both checked and disabled states
-      for (var state in active) {
-        if (active[state]) {
-          on(input, state, true);
-        } else {
-          off(input, state, true);
-        };
-      };
-
-    } else if (!direct || method == 'toggle') {
-
-      // Helper or label was clicked
-      if (!direct) {
-        input.trigger('ifClicked');
-      };
-
-      // Toggle checked state
-      if (active) {
-        if (node[_type] !== _radio) {
-          off(input, state);
-        };
-      } else {
-        on(input, state);
-      };
-    };
-  };
-
-  // Set checked or disabled state
-  function on(input, state, keep) {
-    var node = input[0],
-      parent = input.parent(),
-      label = input.parent().siblings(),
-      remove = state == _disabled ? 'enabled' : 'un' + _checked,
-      regular = option(input, remove + capitalize(node[_type])),
-      specific = option(input, state + capitalize(node[_type]));
-
-    // Prevent unnecessary actions
-    if (node[state] !== true && !keep) {
-
-      // Toggle state
-      node[state] = true;
-
-      // Trigger callbacks
-      input.trigger('ifChanged').trigger('if' + capitalize(state));
-
-      // Toggle assigned radio buttons
-      if (state == _checked && node[_type] == _radio && node.name) {
-        var form = input.closest('form'),
-          stack = 'input[name="' + node.name + '"]';
-
-        stack = form.length ? form.find(stack) : $(stack);
-
-        stack.each(function() {
-          if (this !== node && $(this).data(_iCheck)) {
-            off($(this), state);
-          };
-        });
-      };
-    };
-
-    // Add proper cursor
-    if (node[_disabled] && !!option(input, _cursor, true)) {
-      parent.find('.' + _iCheck + '-helper').css(_cursor, 'default');
-    };
-
-    // Add state class
-    parent[_add](specific || option(input, state));
-    label[_add](specific || option(input, state));
-
-    // Remove regular state class
-    parent[_remove](regular || option(input, remove) || '');
-    label[_remove](regular || option(input, remove) || '');
-  };
-
-  // Remove checked or disabled state
-  function off(input, state, keep) {
-    var node = input[0],
-      parent = input.parent(),
-      label = input.parent().siblings(),
-      callback = state == _disabled ? 'enabled' : 'un' + _checked,
-      regular = option(input, callback + capitalize(node[_type])),
-      specific = option(input, state + capitalize(node[_type]));
-
-    // Prevent unnecessary actions
-    if (node[state] !== false && !keep) {
-
-      // Toggle state
-      node[state] = false;
-
-      // Trigger callbacks
-      input.trigger('ifChanged').trigger('if' + capitalize(callback));
-    };
-
-    // Add proper cursor
-    if (!node[_disabled] && !!option(input, _cursor, true)) {
-      parent.find('.' + _iCheck + '-helper').css(_cursor, 'pointer');
-    };
-
-    // Remove state class
-    parent[_remove](specific || option(input, state) || '');
-    label[_remove](specific || option(input, state) || '');
-
-    // Add regular state class
-    parent[_add](regular || option(input, callback));
-    label[_add](regular || option(input, callback));
-  };
-
-  // Remove all traces of iCheck
-  function tidy(input, callback) {
-    if (input.data(_iCheck)) {
-
-      // Remove everything except input
-      input.parent().html(input.attr('style', input.data(_iCheck).s || '').trigger(callback || ''));
-
-      // Unbind events
-      input.off('.i').unwrap();
-      $('label[for="' + input[0].id + '"]').add(input.closest('label')).off('.i');
-    };
-  };
-
-  // Get some option
-  function option(input, state, regular) {
-    if (input.data(_iCheck)) {
-      return input.data(_iCheck).o[state + (regular ? '' : 'Class')];
-    };
-  };
-
-  // Capitalize some string
-  function capitalize(string) {
-    return string.charAt(0).toUpperCase() + string.slice(1);
-  };
-})(jQuery, 'iCheck', 'checkbox', 'radio', 'checked', 'disabled', 'type', 'click', 'touchbegin.i touchend.i', 'addClass', 'removeClass', 'cursor');

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/js/respond.min.js
----------------------------------------------------------------------
diff --git a/gemfire-site/content/js/respond.min.js b/gemfire-site/content/js/respond.min.js
deleted file mode 100644
index 8353e99..0000000
--- a/gemfire-site/content/js/respond.min.js
+++ /dev/null
@@ -1,6 +0,0 @@
-/*! matchMedia() polyfill - Test a CSS media type/query in JS. Authors & copyright (c) 2012: Scott Jehl, Paul Irish, Nicholas Zakas. Dual MIT/BSD license */
-/*! NOTE: If you're already including a window.matchMedia polyfill via Modernizr or otherwise, you don't need this part */
-window.matchMedia=window.matchMedia||function(a){"use strict";var c,d=a.documentElement,e=d.firstElementChild||d.firstChild,f=a.createElement("body"),g=a.createElement("div");return g.id="mq-test-1",g.style.cssText="position:absolute;top:-100em",f.style.background="none",f.appendChild(g),function(a){return g.innerHTML='&shy;<style media="'+a+'"> #mq-test-1 { width: 42px; }</style>',d.insertBefore(f,e),c=42===g.offsetWidth,d.removeChild(f),{matches:c,media:a}}}(document);
-
-/*! Respond.js v1.1.0: min/max-width media query polyfill. (c) Scott Jehl. MIT/GPLv2 Lic. j.mp/respondjs  */
-(function(a){"use strict";function x(){u(!0)}var b={};if(a.respond=b,b.update=function(){},b.mediaQueriesSupported=a.matchMedia&&a.matchMedia("only all").matches,!b.mediaQueriesSupported){var q,r,t,c=a.document,d=c.documentElement,e=[],f=[],g=[],h={},i=30,j=c.getElementsByTagName("head")[0]||d,k=c.getElementsByTagName("base")[0],l=j.getElementsByTagName("link"),m=[],n=function(){for(var b=0;l.length>b;b++){var c=l[b],d=c.href,e=c.media,f=c.rel&&"stylesheet"===c.rel.toLowerCase();d&&f&&!h[d]&&(c.styleSheet&&c.styleSheet.rawCssText?(p(c.styleSheet.rawCssText,d,e),h[d]=!0):(!/^([a-zA-Z:]*\/\/)/.test(d)&&!k||d.replace(RegExp.$1,"").split("/")[0]===a.location.host)&&m.push({href:d,media:e}))}o()},o=function(){if(m.length){var b=m.shift();v(b.href,function(c){p(c,b.href,b.media),h[b.href]=!0,a.setTimeout(function(){o()},0)})}},p=function(a,b,c){var d=a.match(/@media[^\{]+\{([^\{\}]*\{[^\}\{]*\})+/gi),g=d&&d.length||0;b=b.substring(0,b.lastIndexOf("/"));var h=function(a){return a.replace(/
 (url\()['"]?([^\/\)'"][^:\)'"]+)['"]?(\))/g,"$1"+b+"$2$3")},i=!g&&c;b.length&&(b+="/"),i&&(g=1);for(var j=0;g>j;j++){var k,l,m,n;i?(k=c,f.push(h(a))):(k=d[j].match(/@media *([^\{]+)\{([\S\s]+?)$/)&&RegExp.$1,f.push(RegExp.$2&&h(RegExp.$2))),m=k.split(","),n=m.length;for(var o=0;n>o;o++)l=m[o],e.push({media:l.split("(")[0].match(/(only\s+)?([a-zA-Z]+)\s?/)&&RegExp.$2||"all",rules:f.length-1,hasquery:l.indexOf("(")>-1,minw:l.match(/\(\s*min\-width\s*:\s*(\s*[0-9\.]+)(px|em)\s*\)/)&&parseFloat(RegExp.$1)+(RegExp.$2||""),maxw:l.match(/\(\s*max\-width\s*:\s*(\s*[0-9\.]+)(px|em)\s*\)/)&&parseFloat(RegExp.$1)+(RegExp.$2||"")})}u()},s=function(){var a,b=c.createElement("div"),e=c.body,f=!1;return b.style.cssText="position:absolute;font-size:1em;width:1em",e||(e=f=c.createElement("body"),e.style.background="none"),e.appendChild(b),d.insertBefore(e,d.firstChild),a=b.offsetWidth,f?d.removeChild(e):e.removeChild(b),a=t=parseFloat(a)},u=function(b){var h="clientWidth",k=d[h],m="CSS1Compat"===c.c
 ompatMode&&k||c.body[h]||k,n={},o=l[l.length-1],p=(new Date).getTime();if(b&&q&&i>p-q)return a.clearTimeout(r),r=a.setTimeout(u,i),void 0;q=p;for(var v in e)if(e.hasOwnProperty(v)){var w=e[v],x=w.minw,y=w.maxw,z=null===x,A=null===y,B="em";x&&(x=parseFloat(x)*(x.indexOf(B)>-1?t||s():1)),y&&(y=parseFloat(y)*(y.indexOf(B)>-1?t||s():1)),w.hasquery&&(z&&A||!(z||m>=x)||!(A||y>=m))||(n[w.media]||(n[w.media]=[]),n[w.media].push(f[w.rules]))}for(var C in g)g.hasOwnProperty(C)&&g[C]&&g[C].parentNode===j&&j.removeChild(g[C]);for(var D in n)if(n.hasOwnProperty(D)){var E=c.createElement("style"),F=n[D].join("\n");E.type="text/css",E.media=D,j.insertBefore(E,o.nextSibling),E.styleSheet?E.styleSheet.cssText=F:E.appendChild(c.createTextNode(F)),g.push(E)}},v=function(a,b){var c=w();c&&(c.open("GET",a,!0),c.onreadystatechange=function(){4!==c.readyState||200!==c.status&&304!==c.status||b(c.responseText)},4!==c.readyState&&c.send(null))},w=function(){var b=!1;try{b=new a.XMLHttpRequest}catch(c){b=new
  a.ActiveXObject("Microsoft.XMLHTTP")}return function(){return b}}();n(),b.update=n,a.addEventListener?a.addEventListener("resize",x,!1):a.attachEvent&&a.attachEvent("onresize",x)}})(this);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/js/usergrid-site.js
----------------------------------------------------------------------
diff --git a/gemfire-site/content/js/usergrid-site.js b/gemfire-site/content/js/usergrid-site.js
deleted file mode 100644
index 1a513fa..0000000
--- a/gemfire-site/content/js/usergrid-site.js
+++ /dev/null
@@ -1,50 +0,0 @@
-var start = 40;
-var end = 210;
-
-function setHeaderForScroll(scrollTop) {
-  if ( ($( window ).width() > 768) && ($('header.secondary').length == 0) ) {
-
-    if(scrollTop > start) {
-        opacity = (Math.floor(scrollTop) - start)/end;
-        //console.log(opacity);
-        percent = Math.min(opacity, 1)
-        red = Math.floor(36 + (52-36) * percent);
-        green = Math.floor(129 - (129-73) * percent);
-        blue = Math.floor(166 - (166-94) * percent);
-        blur = Math.floor(2 * percent);
-    } else {
-        opacity = 0;
-        red = 36;
-        green = 129;
-        blue = 166;
-        blur = 0;
-    }
-    $("#home-logo").css("opacity", opacity);
-    $("header").css("box-shadow", "0px 1px "+blur+"px rgb("+red+','+green+','+blue+")");
-  } else {
-    $("#home-logo").css("opacity", 1);
-    $("header").css("box-shadow", "0px 1px 2px rgb(52,73,94)");
-  }
-}
-
-$(document).ready(function() {
-
-    $('table').addClass('table');
-
-    // Detect initial scroll on page load
-    setHeaderForScroll($("body").scrollTop());
-
-    //reduce the opacity of the banner if the page is scrolled.
-    $(window).scroll(function () {
-      setHeaderForScroll($("body").scrollTop());
-    });
-
-    // $(".navbar-toggle").bind("click", function(){
-    //     if($(".collapse").hasClass("collapse"))
-    //         $("#home-logo").css("opacity", 100);
-    //     else
-    //         $("#home-logo").css("opacity", 0);
-    // });
-  
-
-});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/releases/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/releases/index.html b/gemfire-site/content/releases/index.html
deleted file mode 100644
index fb986ff..0000000
--- a/gemfire-site/content/releases/index.html
+++ /dev/null
@@ -1,239 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-
-<head>
-    <meta charset="utf-8">
-    <title>Apache Geode (incubating) — </title>
-    <meta http-equiv="x-ua-compatible" content="ie=edge" />
-    <meta name="viewport" content="width=device-width, initial-scale=1, minimum-scale=1, maximum-scale=1" />
-    <meta name="keywords" content="Apache Geode, Geode, GemFire, In-memory, IMDB, IMDG, cache">
-    <meta name="description" content="Apache Geode is a distributed, in-memory database with strong data consistency, built to support transactional applications with low latency and high concurrency needs." />
-    <meta property="og:title" content="Apache Geode (incubating)" />
-    <meta property="og:description" content="Apache Geode is a distributed, in-memory database with strong data consistency, built to support transactional applications with low latency and high concurrency needs." />
-    <!-- Loading Typekit -->
-    <script type="text/javascript" src="//use.typekit.net/ddl7izx.js"></script>
-    <script type="text/javascript">try{Typekit.load();}catch(e){}</script>
-    <!-- Place this tag right after the last button or just before your close body tag. -->
-    <script async defer id="github-bjs" src="https://buttons.github.io/buttons.js"></script>
-        <!-- Loading Bootstrap -->
-    <link href="/bootstrap/bootstrap.min.css" rel="stylesheet" type='text/css'>
-    <link href="/css/bootflat.css" rel="stylesheet" type='text/css'>
-    <link href="/css/geode-site.css" rel="stylesheet" type='text/css'>
-    <link href="https://fonts.googleapis.com/css?family=Open+Sans:200,400,500,300,600,800,700,400italic,600italic,700italic,800italic,300italic" rel="stylesheet" type="text/css">
-    <link href="/css/font-awesome.min.css" rel="stylesheet" type='text/css'>
-
-    
-
-    <!-- HTML5 shim, for IE6-8 support of HTML5 elements. All other JS at the end of file. -->
-    <!--[if lt IE 9]>
-      <script src="js/html5shiv.js"></script>
-      <script src="js/respond.min.js"></script>
-    <![endif]-->
-</head>
-<body>
-
-    <header class="navbar navbar-inverse navbar-fixed-top bf-docs-nav " role="banner">
-    <div class="container">
-        <div class="navbar-header">
-            <button class="navbar-toggle" type="button" data-toggle="collapse" data-target=".bf-navbar-collapse">
-                <span class="sr-only">Toggle navigation</span>
-                <span class="icon-bar"></span>
-                <span class="icon-bar"></span>
-                <span class="icon-bar"></span>
-            </button>
-        </div>
-        <a href="/" class="navbar-brand">
-                <img id="home-logo" src="/img/apache_geode_logo_white_small.png" />
-            </a>
-        <nav class="collapse navbar-collapse bf-navbar-collapse" role="navigation">
-            <ul class="nav navbar-nav navbar-right">
-                <li class=""><a href="/community/"><span class="icns icon-group"></span></a></li>
-                <li><a href="http://geode.docs.pivotal.io" target="_blank"><span class="icns icon-book"></span></a></li>
-                <li><a href="http://github.com/apache/incubator-geode" target="_blank"><span class="icns icon-github-sign"></span></a></li>
-                <!--<li><a href="https://trello.com/b/exQmJIOn/usergrid" target="_blank"><span class="icns icon-trello"></span></a></li>-->
-                <li><a href="https://issues.apache.org/jira/browse/GEODE/"
-                       target="_blank"><span class="icns icon-bug"></span></a></li>
-                <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank"><span class="icns icon-stackexchange"></span></a></li>
-                <li><a href="/community/#mailing-lists"><span class="icns icon-envelope"></span></a></li>
-                <!-- <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li> -->
-                <li><a href="https://twitter.com/apachegeode" target="_blank"><span class="icns icon-twitter"></span></a></li>
-                <li><a href="https://cwiki.apache.org/confluence/display/geode/" target="_blank"><span class="icns icon-edit"></span></a></li>
-                <li><a href="/releases/"><span class="icns icon-releases"></span></a></li>
-            </ul>
-        </nav>
-    </div>
-    </header>
-
-
-
-
-<section class="bf-tagline">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12">
-    	    	<h2>Apache Geode Releases</h2>
-			</div>
-		</div>
-	</div>
-</section>
-
-<section class="bf-releases">
-    <div class="container">
-    	<div class="row">
-    	    <div class="col-md-12" id="events">
-    	    	<h2 class="icns-download"><span>Releases</span></h2>
-			</div>
-		</div>
-		</br>
-		<div class="row">
-    		<div class="col-md-9 done">
-    			<p>
-				Releases of Geode are made available to the general public at no charge, under the <a href="http://apache.org/licenses/">Apache License</a>, in both binary and source distributions.
-				</p>
-				<p>
-        <ul>
-          <li>Alpha 1.0 Releases - Geode 1.0.0-ALPHA - <strong>N/A</strong></li>
-          <li>General Availability (GA) Releases - Geode 1.0.0 - <strong>N/A</strong></li>
-        </ul>
-        </p>
-        <p>
-					Project releases are approved by vote of the Apache Geode Podling Project Management Committee (PPMC) and Apache Incubator (PMC). Support for a release is provided by project volunteers on the project <a href="http://geode.incubator.apache.org/community/#mailing-lists">mailing lists</a>. Bugs found in a release may be discussed on the list and reported through the <a href="https://issues.apache.org/jira/browse/GEODE">issue tracker</a>. The user mailing list and issue tracker are the only support options hosted by the Apache Geode project.
-				</p>
-				<p>
-					Note: When downloading from a mirror, please be sure to verify that checksums and signatures are correct. To do so, use the checksum and signature files from the main Apache site at <a href="https://dist.apache.org/repos/dist/release/geode/">https://dist.apache.org/repos/dist/release/geode/</a>. Find here the KEYS file, which contains all OpenPGP keys we use to sign releases here: <a href="https://dist.apache.org/repos/dist/release/geode/KEYS">https://dist.apache.org/repos/dist/release/geode/KEYS</a>
-				</p>
-				<p>
-					The PGP signatures can be verified using PGP or GPG. First download the <a href="https://dist.apache.org/repos/dist/release/geode/KEYS">KEYS</a> as well as the <a href="https://dist.apache.org/repos/dist/release/geode/geode-1/v1.0.0/">asc signature</a> file for the particular distribution. Then verify the signatures using:
-				</p>
-				<p>
-       				% pgpk -a KEYS
-					</br></br>
-        			% pgpv ${filename}.tar.gz.asc
-        			</br></br>
-    				or
-    				</br></br>
-        			% pgp -ka KEYS
-        			</br></br>
-        			% pgp ${filename}.tar.gz.asc
-        			</br></br>
-    				or
-    				</br></br>
-        			% gpg --import KEYS
-        			</br></br>
-        			% gpg --verify ${filename}.tar.gz.asc
-        			</br></br>
-        		</p>
-    			<p>
-					Alternatively, you can verify the MD5 signature on the files. A Unix program called md5 or md5sum is included in many Unix distributions. It is also available as part of <a href="http://www.gnu.org/software/textutils/textutils.html">GNU Textutils</a>. Windows users can get binary md5 programs from <a href="http://www.fourmilab.ch/md5/">here</a>, <a href="http://www.pc-tools.net/win32/md5sums/">here</a>, or <a href="http://www.slavasoft.com/fsum/">here</a>.
-				<p>
-					If you want to build directly from the sources, please check the <a href="/docs/getting-up-and-running-locally/">Project Docs</a>.
-				</p>
-			</div>
-		</div>
-	</div>
-</section>
-
-
-
-
-<footer class="bf-footer" role="contentinfo">
-    <div class="container">
-        <div class="row">
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header"><a href="/">Home</a></li>
-                    <li class="nav-header"><a href="/community/">Community</a></li>
-                        <li><a href="/community/#events">Events</a></li>
-                        <li><a href="/community/#mailing-lists">Mailing Lists</a></li>
-                        <li><a href="/community/#deployments">Deployments</a></li>
-                        <!-- <li><a href="/community/#committers">Commiters</a></li> -->
-                </ul>
-            </div>
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header"><a href="http://github.com/apache/incubator-geode" target="_blank">Code</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Project+Proposals+and+Specifications" target="_blank">Specifications</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Geode+Internal+Architecture" target="_blank">Internal Architecture</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Writing+tests" target="_blank">Writing Tests</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Criteria+for+Code+Submissions" target="_blank">Code Submissions</a></li>
-                </ul>
-            </div>
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header">Resources</li>
-                    <li><a href="http://github.com/apache/geode-incubator" target="_blank">GitHub Code</a></li>
-                    <li><a href="http://geode.docs.pivotal.io" target="_blank">Docs</a></li>
-                    <li><a href="https://issues.apache.org/jira/browse/GEODE" target="_blank">JIRA Bug Tracker</a></li>
-                    <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank">StackOverflow</a></li>
-                    <li><a href="/community/#live">Live Chat</a></li>
-                    <li><a href="https://twitter.com/apachegeode" target="_blank">Twitter</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Index#Index-Geodein5minutesGeodein5minutes" target="_blank">Geode in 5 minutes</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/How+to+Contribute" target="_blank">How to Contribute</a></li>
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Application+Development" target="_blank">Application Development</a></li>
-
-                    <li><a href="https://cwiki.apache.org/confluence/display/GEODE/Technology+FAQ" target="_blank">FAQ</a></li>
-
-                </ul>
-            </div>
-            <div class="col-md-2">
-                <ul class="nav nav-list">
-                    <li class="nav-header">Apache</li>
-                    <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                    <li><a href="http://www.apache.org/foundation/sponsorship.html" target="_blank">Sponsorship</a></li>
-                    <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                    <li><a href="http://www.apache.org/security/">Security</a></li>
-                    <li><a href="http://www.apache.org/" target="_blank">Apache Foundation</a></li>
-                </ul>
-            </div>
-            <div class="col-md-4">
-              <a class="twitter-timeline" href="https://twitter.com/search?q=%23ApacheGeode%20OR%20Apache%20Geode" data-widget-id="665030887004725248">Tweets about #ApacheGeode OR Apache Geode</a>
-<script>!function(d,s,id){var js,fjs=d.getElementsByTagName(s)[0],p=/^http:/.test(d.location)?'http':'https';if(!d.getElementById(id)){js=d.createElement(s);js.id=id;js.src=p+"://platform.twitter.com/widgets.js";fjs.parentNode.insertBefore(js,fjs);}}(document,"script","twitter-wjs");</script>
-            </div>
-        <!--
-            <div class="col-md-4">
-                <div class="input-group form-search">
-                    <input type="text" class="form-control search-query">
-                    <span class="input-group-btn">
-                        <button type="submit" class="btn btn-primary" data-type="last">Search</button>
-                    </span>
-                </div>
-            </div> -->
-        </div>
-        <div class="row">
-          <center>
-            <div id="copyright">
-                <a href="http://incubator.apache.org" target="_blank"><img src="/img/egg-logo.png" /></a><br/><br/>
-                <p class="credits">
-                Apache Geode is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by the Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
-                </p>
-                <p>Copyright © 2015 The Apache Software Foundation, Licensed under the Apache License, Version 2.0.<br>
-                Apache, Apache Geode (incubating), and the Apache feather logos are trademarks of The Apache Software Foundation.</p>
-                <p>Site designed & assembled with love by <a href="https://github.com/ryuneeee">@ryuneeee</a> + <a href="https://github.com/realbeast">@realbeast</a> + <a href="https://twitter.com/timanglade">@timanglade</a> + <a href="https://twitter.com/snoopdave">@snoopdave</a> for Apache Usergrid.</p>
-                <p>Modified for Apache Geode by <a href="https://twitter.com/william_markito">@william_markito</a>.</p>
-            </div>
-          </center>
-        </div>
-    </div>
-</footer>
-
-<script type="text/javascript" src="/js/head.js"></script>
-<script type="text/javascript">
-    head.js("/js/jquery-1.10.1.min.js", "/js/bootstrap.min.js", "/js/usergrid-site.js");
-</script>
-<!-- update the code below for google analytics -->
-<!--
- <script>
-  (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
-  (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
-  m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
-  })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
-
-  ga('create', 'UA-45815079-1', 'apache.org');
-  ga('send', 'pageview');
-
-</script> -->
-
-
-
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/static/github-btn.html
----------------------------------------------------------------------
diff --git a/gemfire-site/content/static/github-btn.html b/gemfire-site/content/static/github-btn.html
deleted file mode 100644
index 76a7c55..0000000
--- a/gemfire-site/content/static/github-btn.html
+++ /dev/null
@@ -1,2 +0,0 @@
-
-<html><body><style type="text/css">body{padding:0;margin:0;font:bold 11px/14px "Helvetica Neue",Helvetica,Arial,sans-serif;text-rendering:optimizeLegibility;overflow:hidden}.github-btn{height:20px;overflow:hidden}.gh-btn,.gh-count,.gh-ico{float:left}.gh-btn,.gh-count{padding:2px 5px 2px 4px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff;white-space:nowrap;cursor:pointer;border-radius:3px}.gh-btn{background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fafafa),to(#eaeaea));background-image:-webkit-linear-gradient(#fafafa,#eaeaea);background-image:-moz-linear-gradient(top,#fafafa,#eaeaea);background-image:-ms-linear-gradient(#fafafa,#eaeaea);background-image:-o-linear-gradient(#fafafa,#eaeaea);background-image:linear-gradient(#fafafa,#eaeaea);background-repeat:no-repeat;border:1px solid #d4d4d4;border-bottom-color:#bcbcbc}.gh-btn:hover,.gh-btn:focus,.gh-btn:active{color:#fff;text-decoration:none;text-shadow:0 -1px 0 rgba(0,0,0,.25);border-color:#518
 cc6 #518cc6 #2a65a0;background-color:#3072b3}.gh-btn:hover,.gh-btn:focus{background-image:-webkit-gradient(linear,0 0,0 100%,from(#599bdc),to(#3072b3));background-image:-webkit-linear-gradient(#599bdc,#3072b3);background-image:-moz-linear-gradient(top,#599bdc,#3072b3);background-image:-ms-linear-gradient(#599bdc,#3072b3);background-image:-o-linear-gradient(#599bdc,#3072b3);background-image:linear-gradient(#599bdc,#3072b3)}.gh-btn:active{background-image:none;-webkit-box-shadow:inset 0 2px 5px rgba(0,0,0,.10);-moz-box-shadow:inset 0 2px 5px rgba(0,0,0,.10);box-shadow:inset 0 2px 5px rgba(0,0,0,.10)}.gh-ico{width:14px;height:15px;margin-top:-1px;margin-right:4px;vertical-align:middle;background-image:url(data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAADIAAAAtCAQAAABGtvB0AAAAGXRFWHRTb2Z0d2FyZQBBZG9iZSBJbWFnZVJlYWR5ccllPAAAB7RJREFUWMPt12tQVPcZx/HHGw0VG6yo1Y42YGIbjamT6JhEbc1AUodaJNbnsNwsFRQUsUSQQUEUNILGotFITTA2olVCI7FoiLdquOgEcFBAQS5Z5bLcXFZcdvfs7ZxfX+yqoLvQ6btO+5w3e3bOdz87+9/5n12i/3RGkSf
 NoV/RQppDnjTq3yjYg9O4kg2s50pOY48hg/E+v63NNtXIomww1dRmey+hCUMRywVthDKntKy8rDynNEIp9LEwaDAhL0XWohzRWIRFiEa53HdqK00cjBAEU16N9RD8MRuz4W899GWNYOQgp4FLfopsvJs4Zj79jKbRdPIas6AxURYLUukHzoiJAfqz1bsPsoq38G4+xLu4a+en528GiDzFcfGnuZIOIU0Jorr8SM3JhoKqk6YH9akQJEPSAifIij9vuo930rMYT46kfCxK7g77i+Oi7oh4hejqLvSb6uM0QrxQf8IJsrItv4AorLk/ojDx6NOnwrocF1qlOoRIq+yPWI07x/cK+lYniEI6H0IkSP0RRuys4uWC7LiQzcWvkYtsxYCp/GXhDFlyiuxcwhPDjQORfd7JvoGSM+SCb+lUa8dA5M6cc0slkxMkWpewJXNWfkWA/IRI78z2iUuP0jkujA1l2xqn1W+ApZ9xHL+4mWFUOkH2V0eVn5iR9mlb6VGlAEaK+kalnIypa69n1jouTLs7r6bNbN72/rs1ByEDPUV4C8PIo/Oqcb8TpCE+0LQ6cveRkMKIpmBrhBh7DzMxjP0VlltbHBeYJOvO7mhJMp7VVUl6Y8fD74ho4snNsogXnCAYd/amYMrMunhsW/06bXxXch0RBwni11X4CTlrgmXjhV3HVnec6WvqrWj/hl4vSJUNCCbnA5/CqgDxD5XrGyO061VRbVwRYCysgg8N1gRCpy/vKTO0aaq0tWI19AiiwQfeqiuZFZH3Ay2BlqiefTdU38KbhmqmIB3V0EOPaqRjylDXExEmYBU+wzmcw2dYhaF21P/P//yMpMn0Cr1BC2khvUGv0GQaOUTBY3kNn2Yl93EfK/k0r+Gxg1w+nDzn+17cqyo1tFsNVoOhXVV6ce98X/Kk4c4AV94u6GwbZKg51Gx7JOh4B7s6DFynL6jMsRrsG6QGGvudxXDj2PQF5KhhL+EWQyHtaS+p
 NhSjAAW64pLqPe0KiSHU8ovPEpHLtUoAJhyGL0YTEcENvsiGCdDeixaeYfhFoYuRrL5Xio2Yh+eIiOCKeYhvKU1RM4Tup5jhsctMPYBcmDv3qTUY+de51q8BkyZ2GY0Y8EEp6hkHWjs/ilvFPxqAu69f27I/q4WhaGK3J8/P/7n2HoB9yS/nprz2G3qBvGgGzaTp5PXm4q+2fzAbHwK6Fp9Z/V4qKJWxo0uOWb2aIfRyCqfzCc7jTzhDeMhYvQFRGR2MoI8eB6OuHwbkPAyrXwdY+iqOVP2t+VLrlYYzVScsOqAxkUjKAW5/QS6P3u04hRhmup+OYemZA2/BtmNHNlF36gpzgJkn2Yq4GVa9VQ13ojsJcDA3dxHBXdJIpqQ5diQ8hnHkNtyI0g47QqLLieD2+W3Gym22omwroN9KRCOufewIUZXSWCIxCajea0eiyhgVG4jYTWFwhDDYm+hmjICoGlvRVQJgGlHCZIseDudyEBGmQlZX2JGVPREiJhNFejsh8H4WESZEGlbobYW+1dhBRHR7MZzMvUwiIrHVpLEjgZZYNRHRvnBnyNYzRERxnQxbIYnaKiKidqdI18dERL0VsBekkGNVRESn/ZwhmV8QEW1ofoTIFk0ljSWPU3OdId+nkgd5qMsfI+HGMB37sH9CeJjJMZJ2nP3Y748Pw+w/3cxdolrpZ30P/nK3EyURfr2/N3Ra1HZkcwfj89AHb2PBtZIQy7NERgeC8NbVpQI2dtsK3T+B/CVwoR+3L0avA+IoEVHaXMj6a3bk6DnG+j0YyYvzlnVezPk+URNqp9bqMzqLq7GJiChiK+NQsX3h1wLlWTSy9b3EgMJp2CRftvTZXt3UiBwsISKiEWUHAHGzHakNDrIG9fLzuUEK5fb5CNYcXCnakEM3sAlvEhHxmBCNQrq9xlZggqw3ad6dh1fNyoRQennhr433bUjN4z8bb78uqmUzJttP4Z7dyAjMg1fud0IvHxduBJsZa
 /UrzBF3HyWBxxj7mzHu0bmUBjRfIi8pUuptL9TeseoAUWl9oK2zX+Cp/AaQnmxEROqoGB2Ddxn9Dt+JUkU+SOpmJLYmd0T1EBHxME5jROvUcU8KuMk1QNXJsa+atuG6pV5TAmiK1N/qG4nIxWVW5VFAqsWYfghclXlhJobwj4YYfHLxUnwTI74prnGNhogn8VeMMFPTKfyw//4MT7kbUJX+bim9VBSuKQI0RZqiviZ6yd9fVQLI3Xj6HoRJzedj+hiCng/E5mxsYCTWxTeGGvmAoGOs0929gJ/S042nXA1Yxbr8qhPtpUDblY5r5od1+VYDIN/CNHp2MEl3NKsl0MpgCDIj2L74gVJWi/bY4wUc2IzGh7DdfiXAorV/gUXsgRs5HjyHKPXl3MbknpVGAYIcbkzuyW1UX8EauJLTwXjEohAqyJDQhkLEYjwNPnDHcmTgS1zGZfwdGVgOd/pvmX8Bbv8r+TZ9z+kAAAAASUVORK5CYII=);background-repeat:no-repeat;background-position:0 0}.gh-btn:hover .gh-ico,.gh-btn:focus .gh-ico,.gh-btn:active .gh-ico{background-position:-25px 0}.gh-count{position:relative;display:none;margin-left:4px;background-color:#fafafa;border:1px solid #d4d4d4}.gh-count:hover,.gh-count:focus{color:#4183c4}.gh-count:before,.gh-count:after{content:' ';position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.gh-count:before{top:50%;left:-3px;margin-top:-4px;
 border-width:4px 4px 4px 0;border-right-color:#fafafa}.gh-count:after{top:50%;left:-4px;z-index:-1;margin-top:-5px;border-width:5px 5px 5px 0;border-right-color:#d4d4d4}.github-btn-large{height:30px}.github-btn-large .gh-btn,.github-btn-large .gh-count{padding:3px 10px 3px 8px;font-size:16px;line-height:22px;border-radius:4px}.github-btn-large .gh-ico{width:22px;height:23px;background-position:0 -20px}.github-btn-large .gh-btn:hover .gh-ico,.github-btn-large .gh-btn:focus .gh-ico,.github-btn-large .gh-btn:active .gh-ico{background-position:-25px -20px}.github-btn-large .gh-count{margin-left:6px}.github-btn-large .gh-count:before{left:-5px;margin-top:-6px;border-width:6px 6px 6px 0}.github-btn-large .gh-count:after{left:-6px;margin-top:-7px;border-width:7px 7px 7px 0}@media(-moz-min-device-pixel-ratio:2),(-o-min-device-pixel-ratio:2/1),(-webkit-min-device-pixel-ratio:2),(min-device-pixel-ratio:2){.gh-ico{background-image:url(data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAGQAAABaCAQAA
 ADkmzsCAAAAGXRFWHRTb2Z0d2FyZQBBZG9iZSBJbWFnZVJlYWR5ccllPAAAE81JREFUeNrtnGl0VFW2gHcIIggINLQoaj9bQHmgjUwRBZMK2A4Iora7CAFjGBIRFESZmwZkEgkiAg0oiigIggJhkkGgAjIpgyAkEAhICCGQkEDmoaru937UkKqQhFTwvbd6Lc5dK6tycm/t8917zj57uhH5/2h+Uk+aSGt5UoIkSJ6UVtJY6omf/Ec1P7lPnhBTKUd7afQHwqi//l1n6V69rHa16SXdox9pZ63yB319LWknplqdFgw78V32EdsV7Nhsadm/xn07793qwWKSdlLrj4CoqkP0vFLKcVYHaNWbFnCXBNbpvHNOYQqltIILP86s01kC5c83i/GYHncMO6Rg9JlPT648tSJ+wclRZ0MKnTDHtOVNCWgoQWP655x1jjub1UzkbQYzibXkODvPjO4nQXLXzWD00AJFGXZ5128FO7EUHwU7Y469m6oomq+vVlpAbQn8/n17EYARQ1eqe/6R6nQ3fgKwF64YL4FSu7IYvdSmvFawNRYLFn5gIn14hVfoyxQ2YcGyNbZ3oaI2NVdKQBUJiJ5s2IErW0dIkLSQO0Skhtwp9aSWVJWa8qgEbR7JVTDs302QAKnMqtQ2WqhE5p3fn7onYx5PUM3rblWjw5UFF/ad2x+Zp2iBtq6EiPsnRBpFwBkefOXFNi+ISQKlo4fGChJT+25hr9KEM2AvGhch9/uOcbvGK+FF5/aztu9hten32kz9tLE+oZ21ldbT5rpR7eFxrD+3P6xI0RN6u68q976gnCQglSYiGQcNe9LOt8OqBvcLnTZo3rtjI9p3G/p6yn7DyDwuQhOuQE7ifUE+q2IdppiN/UdYxj3mK4qihXrNQ2PZFMV8jXtZtv+IGUXf9VFEg93zATtPi0jVoqsAdqs1p1hjGXYAa7bUFeFpDPjp31LfN4zbNEWJusga7hXpf7VU5YsSni3Cva
 ydnqLoRb3NFxl/aVGYDnwhIiJ/zU2ijJafKgEiInwJhVf+0tw3kO6K2Ti/jzYiemf/3LJAzIaaRGiTuM+Mol19kbHmPcDOgyIi7TrnpZQFYthnvyM1RWiMAd8P9Qmkx+fKqAxGiIjolLIwFEVPqJ8II4dmKT0W+iLjzHoo2OX4fGQJ5bScxNr1RUSKDkPCWp9AwuKVpQncIyJi/r1cEPRRERotPquExfsiI/M0ZI91fM67SLlt21MiItkTIfOUTyCh+crm1Y7PZnv5ID26iIhs3aiE5vsiw5YLSS87PjuWddkt6RURkaRXwJrj2xpB2T7C8TnkBiDj+omI7PinovgiA2DV03Kn1JXaRmH5IGfNUltqf/cMgM8gS8Icn/vnlw/ydR8RkaWvVwZkyUtyp9SWWrYL5YMc6iS1pdZXL/sM0tuqvDNe22ugthuXWh6G2Vg4QFtr2yETld5WX2TYc+DgVNoTSDvWlcth5yla0/bQh2DP8glkSLbyxpcaoK211br9ZqNskLHp0/poW23Zf5kyJNsXGUXHIHbl+adovTco8Q1s5YBs4mnang04tRaKfvMJZPp5JfIozfkbzZiyKa6XrXSMoZnpP/E3mvJwRKwyI9GnJ/I5pB6SZiJyhwT88h7ZZWD8jMMXaZZ2FPjUJ5Aftihm49tnaDr1tc9G2Xek714VP/5KZL7ZCdDT/nZ2VErMMXsMH9KGh7/uZDaUzZt9WiPdwTAiekldOiV3rx4c0S59aMGm/GQM53wqLDjBIrrjsHjrRvQyDKCbTyB5I/sUKrpYRB/SuMHr+QELlo1xLpDwwkt7sWBhPnVFRHSx0rewYIRPINVIgbObpUPCI8RdWu6weNdOdYEUpQ99yn3y7fLk2c3ARXwyg4QOSxMUNTSYVitD1PranLXDNi3vm6soDnW84BAj6ICfiIgGq6EsS+BJ36xGRgDGnKHyeEIbrGkLvjBv7J+fCmAUASTMcp5YQx6fMxQDGOajYUr
 VgjUDchVNXRrA4rF71VBDDWVMujL1Ur+CAVlhi9yq+j69rLyZW7AaH/13biceiq6azdIh8ysMDAzI3A1X1hWk5p+9uMzp03d8VYsygJP46iqIEHLsYIhd0VNLA23b5yzvu3HAuhD71EvKzAv988ddGbXNidFYzygh9uMH6eG7Z0U7CiE36fWedTrv/yBvFYvsRWnr4dLy/EsZO5OXSwN5TEz9QvOSgULaVMJ54zaWbIozG4qmL1nCDnawo7d1bJwy4ee+eaOS/rVbRER76lXFbGyJ5WsfZ69LTi/sYM1cNVFMYpKO1pyLmyB5eX5a6u74aDGJadUkWxZgI6SSHjvN+HFrbIhNUfrHbfiqcFSobfRRZdye3kXDTg87rN11p6KE2LYd50ceqmz8gR4UAFw9snB4nc62gnPbID7ampOyN3HH0n9m/OpwSqh8gEOEp9kRe3BglnPXuKYMuGBm2OEe9ogrrp1kUNaJA2yn081EhGjNcafKzYLMExiJOwxr3ln3TnKMx24yqkUwW4t2rjzdJ7u07bBP1venbDFsIehmY3RUYzDnS90OExnEzQcBRWjKl1hsMXuPfnJ2aGZYvqJGeOGQ1LlJ+4/YYrCwiCZ/TNwUf55hFj+TChhcZi8z6Yz/Hxb3pSqvsMIzOOc+VvDSHyjo/6JRhba8xXzWYGEHa5jLQFpTRW61W+1Wu9VutVvtVvtfbf5SXx6URyVAOkqgBEoHCZBH5EH5k/zH2BJ+0kAekcBSs+4mMUmgtJD6f0juXWtpF/1A1+kJzdBCLdB0jdNonaLPaM2b/vKGEiAmMT3a5cuRR79J2ZuTaM2yW+1FRVk555J3H1m6cPjDz4lJTNLu5rK8VfRFXeXI9JZ65OlK7VrpQoKa0kpM1YOXjEne5cj0lhp2LEyyLB5dPVhM0koqc+PUT3tp3A1SDI7juIao74++kQRWDY6ekpNIBVrWuVUTqwZLoDTyFaOF/lRywD3tkXlDsgdnR+aV
 ErHfqS18WhdNxTS8b/qx6zNvnOEwv3LG4RB7tvSj74aLSZr6sF40Uj1i8q9Zo1I2x17YZ49xeSb2mKR9P8RNT+lt9UDJ1YgKY7QQ09aP7J7JhQwW0ZMHil0FqvBXevMl1zymWcHWGWKS5hVCUX+dXTy8t3I2xRW6aiC2sIzPWMgytrrqITbGDczxgJldofXyUK1OJ6M9IH6jV9kRLKrzmsvHBzgZXauTPFQRjGWuYb1eFH3SHoOF9YygM3fjvg/4cQ9/ZyQbsNhj1sSHFblRvtEb6f17a3VKsrjHlUY/bnh/qUJ/0lyXnLfU6iT33ghknmtIYzLS9mBhEU+XHcGiGs+wGEvanjEZbpR55QqoJYHxxU9jy9Tm0lYelnrlTsT60kLaj3mMLa7LTq29QaWKvukazsxkWwzRvFCBu+VHV9baYmYmu1HeLGdQbbfPcmPMw18ecW57baSuiPhLbakvDaWRNJQGUlP8pI60dZ7REn/muS7dMVvalrlStKVrx5iThIWoAeF6RL/QTuXuM930O02MfIsoLHOTnCAFWlZcqtHYCLvVOZaPREQ2js5MSNj476HOTS/oul3dVD148eikmLzLu6JERIhyLnvruIgyVLH662HHQCZfNiy8RxVd5RzYQQ0U0ZraVrvpaxqpvfRFfVRv00A94jxjE1V4z7BMuez8/XCpK6VK7Q6Zp50Yyx3POiXG8eu1+FmDxfTwc++/8dWYtVO3zoievGTM8L71n/5osOuKtIPO57/c8XvmmXodSq0e0n6OQbyZm7OLt0REwhLck8XQWLWW2DkK1J2i65UmIsKgvF0DXVUTpanihltnODHicO7ReaeLSx6yfi+ZtrYXubInUJDsnMp3EOvo+XGmNLweo6omKIqZw4cZ57hbfa5WaF9HCctx3q1/HTnkzEAmarWSMv7SxpENwU57V19hMhVsRVfFWaZGAHaAvEv3t70eRB1DmnaJr6nh6BuaUlGQwRlunb94uuuqn
 iVEVFszyTmmL919ddOPVBTk2ilp41refO7oi54sJW+X+QdH8vn3/Tzi6puaUFGQ8AK9zymiReK+HoaimEtmGBte+gUAK43dfW3P/FDhJ3Ktp9k1lfgrVoDUgyUml9Yz2xRl7BVGu/sCy0tTX3cccC1vRo5PUxSzXb1qrfq3NwwAY527q/bsd25UzOH1TOIbuOv2jGgAw4jwTv/py47hbDnOfe6+Az5geEwlGm37zdnzD08Z28Y4x+POfNS4P/MUPrUNE92710uOHss/vUB6z3VMrLRZboxHfcTwmEoZMxzPsvd8TxmnvwPAxp2unmXd8LGlHnApXGobVoAzq7xA+u9XlCHZBLtB3vIVJMRdB0Hg0CxF6fOrp4yMIwB5R4t7Tk7yFaQos9iDz/sVIMO7MiI8TVGmpuC2XwbM9RVEUZd6vGNaiqK8fsVTRt5lgGvfFfdcXIDvzW0lZ6wAyE/zAulVoCizDxf3jFlVCRC3Izr3gKKEFnjKsOYCXJxR3JO+sBIg7lud8iGALc9b+RqKMttDYU5e5ztIcaXw3I2ONedlXAKQMKm4J2u67xwea25CyR4RcWj+qJXFPXOW+ooRZi0uEJ/xTVkgh6ZLA2kgDaWh/ClxpK8YthxpIHdJfblL7v55SikgYVZFGe+hAX6Y7CvI0Mziq8evVErWc9lyAI5/KjWlljSQ+lL/QBdfQfKPSSOpL3+WBlL32AIAe64XyBt5ihIZqy/pSxqmofr8x7NCbb6BjErV7mrWLhqi4RGxihLpVfNoTQZIO3S+Z7rZ9hqhPEcfcn0k2UZ3zHQh5FpE6mEA6yUvkDGXFaVvkjbXlvqidtUXJg6efNk3kBlHNVK76qv6sgb1vaAoI7y0VuE+gMzT6zvSkhfpygu8zAofQT4mkm68SvdfXsk8A1D4sxfIxyccc/rzQds1swudeZxns38ckFdxjDHpRNEBE4/TaVcfR3nUTK9yWttcAMP2RS8edDnP1OW0Dxjbi/
 3VMc87DHybt2O9drVzng+jMU/yBO15ivEpe9/JqhjGiKsZuxlIV54giKcmjHL0Rq/3WuyvOkazcpw4rOu7pJ00TXyQgxXE2EUD95fVcFvS3qU9F4c59FafXdzjqjvgDpbYYtaeHHatfOPxnaz1J+wxRHkYPFsdz/fCKC+Q+o46xot7pJkz/t5cgqT17Nvpxx7KNx4PEe6VHG+WvMfp2Xi/wkTHsVecte9Nnd5JrH6y8iEWYMFyee/6E7OSR5Zws8ZkzL6w4cSFfViw8EmxBaWNHSXQY9MJ9LbjjS0OizUyVO4UoQexyUuDusnD4idCI8Jzvkj7tYRtdShrIeE8UMIhqOMsE4StJSMhtX90WaxLRES0pn6rNv15zJ10YS47sGB5v0QZ7ftphiNs9ynPecZaXHGxLceL4ZxSQp3lyZslQPypxQps1+KaPSuPSUOpJ40kIHmXN0jyrtsfKiWTEnDWFRjqdd1fi6Y7VLAa+qQIJhYPO6RW/VyriFCf56LnXz+pVs/jWe4u4WmaHJ58ZF7R9FKiYOcdz+SDgdJcBD++MWwJG6oHS5AEStDC4dfPqfXX+/7NPxrs9OR/LyXiRtC6E84BxmtNqjMu7adQq9p0p4bq3/XN4ri8R1Rx1nUOc0096fjb2pPFlrSHlAjX+whNnpUmIjQk17CnHVkzacGwHz/OOecOOlx1V8kvLfEVTZs86z7vjdLCbP62ZUNcOmqt+ovwr3nnFLWrVfMc7/OMTe9lU5acUULsY9OVyM3XJSKWO75hSLZteWnlN/hz2FnNtKNqsDQTP6IAu2EzChyqIGe7vQguTAXI3w5p673Cew9XDU7c5sQ4WkY5FM+fPNDTlS6Yr37UK9gyLs1zKn17WlG+ilOU1fHK8AMlMJzh1hD7yQN0KSMu2cqVLohdWTVYWs6rx3qvcq1xABcmApwb7gVSTVpWDT65xnliIa3KDhR/tjrePeyv9TbewLLv13mJ05M++31IlrJoi6L
 MXKQoK9cro496hZO+cF27Kp7Pyq4kYpD7nYRNdTpLR7nH+gxRfM7k3Fj4fRS4fp5+0w3iJ/dIhzqdEza4iQeVF8VtzJZZxRFcy1tNmOrKiEy9pER9pigffaEos2d4gmgjtbium5XMVo84SWly3BHc1MNms5ikndwtVURSN8CZ0d4glzZKFblbAsTU7R+ph4ujxjcKSHezxUy75Ea5pv0L2jGA4fQbf1r5cL7i+jljigtE/TVC013XTEuxxdD9BlL8XWFPsOZsiqoeLCZ5Sv47aQs4TPvL7wHED4Rz26SjmKoHb55RlOnGWF6B8jfescfMvuCxMo5pmNYQGXXUjTDHBfLeCa2h4Z55xtlJ9hjeuXGmB3/meOQHz6yf+sCzYkrcDo5Y/a6JAGsmQfKeB57dMK1YnwGzK1QARxVGY4k+6WXEZ+s3YdnKrFmK8vV4RZn6kaKGZhafFWpbexILoytaZ0ckeR4uU965bYXpsGEawPz3ADZFAYbV09TPpX+F84f48TaW07+MuC7ya7YrZsITSrO9Rl5N+BkLb+NDdpcW7Lr+5T3AuHbKMEqxuGLw7a1EEV5gs2HZEuuVHyzzeCtna6xhYXNZKrfcm9aTuArZvsfpQWWqH3iAT7DYY2J+m5Ra9utjofbJl3cfNSxY+Jj/qlzVAFXoxvfXJ6PdLY8VdKHyJRz40YnFWLDk7Np99NPECWkDc18vCrWH2sKLBuW8n7bw3N6jebuwYGERwdxkrQi1eJ4PiCaONPLIJZXjrGYyz3DzZSIi+PEkE1zJ6FKOzYwngP+U/5xBDQKIYDKLiWYzm1nDl0ykH229/0PArXarlWz/A3bbfoDcyFIFAAAAAElFTkSuQmCC);background-size:50px 45px}}</style> <span class=github-btn id=github-btn> <a class=gh-btn id=gh-btn href="#" target=_blank> <span 
 class=gh-ico></span> <span class=gh-text id=gh-text></span> </a> <a class=gh-count id=gh-count href="#" target=_blank></a> </span> <script type="text/javascript">var params=function(){var d=[],c;var a=window.location.href.slice(window.location.href.indexOf("?")+1).split("&");for(var b=0;b<a.length;b++){c=a[b].split("=");d.push(c[0]);d[c[0]]=c[1]}return d}();var user=params.user,repo=params.repo,type=params.type,count=params.count,size=params.size,head=document.getElementsByTagName("head")[0],button=document.getElementById("gh-btn"),mainButton=document.getElementById("github-btn"),text=document.getElementById("gh-text"),counter=document.getElementById("gh-count");function addCommas(a){return String(a).replace(/(\d)(?=(\d{3})+$)/g,"$1,")}function jsonp(b){var a=document.createElement("script");a.src=b+"?callback=callback";head.insertBefore(a,head.firstChild)}function callback(a){if(type=="watch"){counter.innerHTML=addCommas(a.data.watchers)}else{if(type=="fork"){counter.innerHTML=addC
 ommas(a.data.forks)}else{if(type=="follow"){counter.innerHTML=addCommas(a.data.followers)}}}if(count=="true"){counter.style.display="block"}}button.href="https://github.com/"+user+"/"+repo+"/";if(type=="watch"){mainButton.className+=" github-watchers";text.innerHTML="Star";counter.href="https://github.com/"+user+"/"+repo+"/stargazers"}else{if(type=="fork"){mainButton.className+=" github-forks";text.innerHTML="Fork";counter.href="https://github.com/"+user+"/"+repo+"/network"}else{if(type=="follow"){mainButton.className+=" github-me";text.innerHTML="Follow @"+user;button.href="https://github.com/"+user;counter.href="https://github.com/"+user+"/followers"}}}if(size=="large"){mainButton.className+=" github-btn-large"}if(type=="follow"){jsonp("https://api.github.com/users/"+user)}else{jsonp("https://api.github.com/repos/"+user+"/"+repo)};</script></body></html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/Rules
----------------------------------------------------------------------
diff --git a/gemfire-site/website/Rules b/gemfire-site/website/Rules
index 3eb6e86..e1173df 100644
--- a/gemfire-site/website/Rules
+++ b/gemfire-site/website/Rules
@@ -1,4 +1,21 @@
+
 #!/usr/bin/env ruby
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
 
 # A few helpful tips about the Rules file:
 #
@@ -13,7 +30,7 @@
 #   item, use the pattern “/about/*/”; “/about/*” will also select the parent,
 #   because “*” matches zero or more characters.
 
-passthrough /\/(bootstrap|css|font|img|js|static)\/.*/ 
+passthrough /\/(bootstrap|css|font|img|js|static)\/.*/
 
 compile '/docs/*' do
   @docs = true

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/build.sh
----------------------------------------------------------------------
diff --git a/gemfire-site/website/build.sh b/gemfire-site/website/build.sh
index 603a5cb..00a9ed0 100755
--- a/gemfire-site/website/build.sh
+++ b/gemfire-site/website/build.sh
@@ -1 +1,18 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
 nanoc compile

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/content/community/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/community/index.html b/gemfire-site/website/content/community/index.html
index affc8e7..a2ec4e9 100644
--- a/gemfire-site/website/content/community/index.html
+++ b/gemfire-site/website/content/community/index.html
@@ -1,3 +1,19 @@
+<!-- Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License. -->
 
 <!-- <div id="map-canvas" style="width: 100%;"></div> -->
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/content/css/geode-site.css
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/css/geode-site.css b/gemfire-site/website/content/css/geode-site.css
index 88bcc48..987c695 100644
--- a/gemfire-site/website/content/css/geode-site.css
+++ b/gemfire-site/website/content/css/geode-site.css
@@ -1,3 +1,19 @@
+/*Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.*/
 @import url("http://fonts.googleapis.com/css?family=Open+Sans:400,300,600,800,700,400italic,600italic,700italic,800italic,300italic");
 /* @import url("http://fonts.googleapis.com/css?family=Source+Sans+Pro:200,300,400,700"); */
 /* Key scaffolding

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/content/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/index.html b/gemfire-site/website/content/index.html
index 5e6a474..ad9858c 100644
--- a/gemfire-site/website/content/index.html
+++ b/gemfire-site/website/content/index.html
@@ -1,6 +1,22 @@
 ---
 title: Performance is key. Consistency is a must.
 ---
+<!-- Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License. -->
 
 <section class="bf-masthead" id="content" role="main">
     <div class="bf-masthead-bg">


[41/50] [abbrv] incubator-geode git commit: GEODE-651: add Chunk unit tests This closes #60

Posted by je...@apache.org.
GEODE-651: add Chunk unit tests
This closes #60


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

Branch: refs/heads/feature/GEODE-14
Commit: fdec70bcd46c568637f0dad976f3778625858608
Parents: 8c9af2a
Author: Sai Boorlagadda <sb...@pivotal.io>
Authored: Thu Dec 10 17:33:57 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Dec 15 16:50:39 2015 -0800

----------------------------------------------------------------------
 .../offheap/AbstractStoredObjectTestBase.java   |   2 +-
 .../offheap/ChunkWithHeapFormJUnitTest.java     |  64 ++
 .../offheap/GemFireChunkFactoryJUnitTest.java   | 129 +++
 .../internal/offheap/GemFireChunkJUnitTest.java | 924 +++++++++++++++++++
 .../offheap/GemFireChunkSliceJUnitTest.java     |  72 ++
 .../internal/offheap/StoredObjectTestSuite.java |  33 +
 6 files changed, 1223 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java
index a5fac48..24cb2fd 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/AbstractStoredObjectTestBase.java
@@ -51,7 +51,7 @@ public abstract class AbstractStoredObjectTestBase {
         Object regionEntryValue = getValue();
         StoredObject storedObject = createValueAsSerializedStoredObject(regionEntryValue);
 
-        Integer actualRegionEntryValue = (Integer) storedObject.getValueAsDeserializedHeapObject();
+        Object actualRegionEntryValue = storedObject.getValueAsDeserializedHeapObject();
         assertEquals(regionEntryValue, actualRegionEntryValue);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapFormJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapFormJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapFormJUnitTest.java
new file mode 100644
index 0000000..bc32367
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/ChunkWithHeapFormJUnitTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class ChunkWithHeapFormJUnitTest extends GemFireChunkJUnitTest {
+
+  @Test
+  public void getRawBytesShouldReturnCachedHeapForm() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    byte[] valueInBytes = getValueAsByteArray();
+    ChunkWithHeapForm heapForm = new ChunkWithHeapForm(chunk, valueInBytes);
+
+    assertNotNull(heapForm);
+
+    assertSame(valueInBytes, heapForm.getRawBytes());
+  }
+
+  @Test
+  public void getChunkWithoutHeapFormShouldReturnGemFireChunk() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    byte[] valueInBytes = getValueAsByteArray();
+    ChunkWithHeapForm heapForm = new ChunkWithHeapForm(chunk, valueInBytes);
+
+    Chunk chunkWithOutHeapForm = heapForm.getChunkWithoutHeapForm();
+
+    assertNotNull(chunkWithOutHeapForm);
+    assertEquals(GemFireChunk.class, chunkWithOutHeapForm.getClass());
+
+    assertEquals(chunk, heapForm.getChunkWithoutHeapForm());
+
+    assertEquals(chunk.getMemoryAddress(), chunkWithOutHeapForm.getMemoryAddress());
+    assertArrayEquals(chunk.getRawBytes(), chunkWithOutHeapForm.getRawBytes());
+    assertNotSame(valueInBytes, chunkWithOutHeapForm.getRawBytes());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactoryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactoryJUnitTest.java
new file mode 100644
index 0000000..d12b823
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkFactoryJUnitTest.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.mock;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GemFireChunkFactoryJUnitTest {
+
+  private MemoryAllocator ma;
+
+  @Before
+  public void setUp() {
+    OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
+    OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
+    LogWriter lw = mock(LogWriter.class);
+
+    ma = SimpleMemoryAllocatorImpl.create(ooohml, stats, lw, 1, OffHeapStorage.MIN_SLAB_SIZE * 1, OffHeapStorage.MIN_SLAB_SIZE);
+  }
+
+  @After
+  public void tearDown() {
+    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  private GemFireChunk createChunk(Object value) {
+    byte[] v = EntryEventImpl.serialize(value);
+
+    boolean isSerialized = true;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    return chunk;
+  }
+
+  @Test
+  public void factoryShouldCreateNewChunkWithGivenAddress() {
+    GemFireChunk chunk = createChunk(Long.MAX_VALUE);
+
+    ChunkFactory factory = new GemFireChunkFactory();
+    Chunk newChunk = factory.newChunk(chunk.getMemoryAddress());
+
+    assertNotNull(newChunk);
+    assertEquals(GemFireChunk.class, newChunk.getClass());
+
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void factoryShouldCreateNewChunkWithGivenAddressAndType() {
+    GemFireChunk chunk = createChunk(Long.MAX_VALUE);
+
+    ChunkFactory factory = new GemFireChunkFactory();
+    Chunk newChunk = factory.newChunk(chunk.getMemoryAddress(), GemFireChunk.TYPE);
+
+    assertNotNull(newChunk);
+    assertEquals(GemFireChunk.class, newChunk.getClass());
+
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+    assertThat(newChunk.getChunkType()).isEqualTo(GemFireChunk.TYPE);
+
+    chunk.release();
+  }
+
+  @Test
+  public void shouldGetChunkTypeFromAddress() {
+    byte[] v = EntryEventImpl.serialize(Long.MAX_VALUE);
+
+    boolean isSerialized = true;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    ChunkFactory factory = new GemFireChunkFactory();
+    ChunkType actualType = factory.getChunkTypeForAddress(chunk.getMemoryAddress());
+
+    assertEquals(GemFireChunk.TYPE, actualType);
+
+    chunk.release();
+  }
+
+  @Test
+  public void shouldGetChunkTypeFromRawBits() {
+    byte[] v = EntryEventImpl.serialize(Long.MAX_VALUE);
+
+    boolean isSerialized = true;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    int rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4 /* REF_COUNT_OFFSET */);
+
+    ChunkFactory factory = new GemFireChunkFactory();
+    ChunkType actualType = factory.getChunkTypeForRawBits(rawBits);
+    assertEquals(GemFireChunk.TYPE, actualType);
+
+    chunk.release();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
new file mode 100644
index 0000000..20cb8dc
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
@@ -0,0 +1,924 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.compression.Compressor;
+import com.gemstone.gemfire.internal.DSCODE;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.cache.BytesAndBitsForCompactor;
+import com.gemstone.gemfire.internal.cache.CachePerfStats;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.internal.cache.RegionEntryContext;
+import com.gemstone.gemfire.internal.offheap.MemoryBlock.State;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
+
+  private MemoryAllocator ma;
+
+  static {
+    ClassLoader.getSystemClassLoader().setDefaultAssertionStatus(true);
+  }
+
+  @Before
+  public void setUp() {
+    OutOfOffHeapMemoryListener ooohml = mock(OutOfOffHeapMemoryListener.class);
+    OffHeapMemoryStats stats = mock(OffHeapMemoryStats.class);
+    LogWriter lw = mock(LogWriter.class);
+
+    ma = SimpleMemoryAllocatorImpl.create(ooohml, stats, lw, 3, OffHeapStorage.MIN_SLAB_SIZE * 3, OffHeapStorage.MIN_SLAB_SIZE);
+  }
+
+  @After
+  public void tearDown() {
+    SimpleMemoryAllocatorImpl.freeOffHeapMemory();
+  }
+
+  @Override
+  public Object getValue() {
+    return Long.valueOf(Long.MAX_VALUE);
+  }
+
+  @Override
+  public byte[] getValueAsByteArray() {
+    return convertValueToByteArray(getValue());
+  }
+
+  private byte[] convertValueToByteArray(Object value) {
+    return ByteBuffer.allocate(Long.SIZE / Byte.SIZE).putLong((Long) value).array();
+  }
+
+  @Override
+  public Object convertByteArrayToObject(byte[] valueInByteArray) {
+    return ByteBuffer.wrap(valueInByteArray).getLong();
+  }
+
+  @Override
+  public Object convertSerializedByteArrayToObject(byte[] valueInSerializedByteArray) {
+    return EntryEventImpl.deserialize(valueInSerializedByteArray);
+  }
+
+  @Override
+  public GemFireChunk createValueAsUnserializedStoredObject(Object value) {
+    byte[] valueInByteArray;
+    if (value instanceof Long) {
+      valueInByteArray = convertValueToByteArray(value);
+    } else {
+      valueInByteArray = (byte[]) value;
+    }
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    return createChunk(valueInByteArray, isSerialized, isCompressed);
+  }
+
+  @Override
+  public GemFireChunk createValueAsSerializedStoredObject(Object value) {
+    byte[] valueInSerializedByteArray = EntryEventImpl.serialize(value);
+
+    boolean isSerialized = true;
+    boolean isCompressed = false;
+
+    return createChunk(valueInSerializedByteArray, isSerialized, isCompressed);
+  }
+
+  private GemFireChunk createChunk(byte[] v, boolean isSerialized, boolean isCompressed) {
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(v, isSerialized, isCompressed, GemFireChunk.TYPE);
+    return chunk;
+  }
+
+  @Test
+  public void chunkCanBeCreatedFromAnotherChunk() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    GemFireChunk newChunk = new GemFireChunk(chunk);
+
+    assertNotNull(newChunk);
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void chunkCanBeCreatedWithOnlyMemoryAddress() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    GemFireChunk newChunk = new GemFireChunk(chunk.getMemoryAddress());
+
+    assertNotNull(newChunk);
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void chunkSliceCanBeCreatedFromAnotherChunk() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    int position = 1;
+    int end = 2;
+
+    GemFireChunk newChunk = (GemFireChunk) chunk.slice(position, end);
+
+    assertNotNull(newChunk);
+    assertThat(newChunk.getClass()).isEqualTo(GemFireChunkSlice.class);
+    assertThat(newChunk.getMemoryAddress()).isEqualTo(chunk.getMemoryAddress());
+
+    chunk.release();
+  }
+
+  @Test
+  public void fillSerializedValueShouldFillWrapperWithSerializedValueIfValueIsSerialized() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    // mock the things
+    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+
+    byte userBits = 0;
+    byte serializedUserBits = 1;
+    chunk.fillSerializedValue(wrapper, userBits);
+
+    verify(wrapper, times(1)).setChunkData(chunk, serializedUserBits);
+
+    chunk.release();
+  }
+
+  @Test
+  public void fillSerializedValueShouldFillWrapperWithDeserializedValueIfValueIsNotSerialized() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    // mock the things
+    BytesAndBitsForCompactor wrapper = mock(BytesAndBitsForCompactor.class);
+
+    byte userBits = 1;
+    chunk.fillSerializedValue(wrapper, userBits);
+
+    verify(wrapper, times(1)).setChunkData(chunk, userBits);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getShortClassNameShouldReturnShortClassName() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getShortClassName()).isEqualTo("GemFireChunk");
+
+    chunk.release();
+  }
+
+  @Test
+  public void chunksAreEqualsOnlyByAddress() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    GemFireChunk newChunk = new GemFireChunk(chunk.getMemoryAddress());
+    assertThat(chunk.equals(newChunk)).isTrue();
+
+    GemFireChunk chunkWithSameValue = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.equals(chunkWithSameValue)).isFalse();
+
+    Object someObject = getValue();
+    assertThat(chunk.equals(someObject)).isFalse();
+
+    chunk.release();
+    chunkWithSameValue.release();
+  }
+
+  @Test
+  public void chunksShouldBeComparedBySize() {
+    GemFireChunk chunk1 = createValueAsSerializedStoredObject(getValue());
+
+    GemFireChunk chunk2 = chunk1;
+    assertThat(chunk1.compareTo(chunk2)).isEqualTo(0);
+
+    GemFireChunk chunkWithSameValue = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk1.compareTo(chunkWithSameValue)).isEqualTo(Long.signum(chunk1.getMemoryAddress() - chunkWithSameValue.getMemoryAddress()));
+
+    GemFireChunk chunk3 = createValueAsSerializedStoredObject(Long.MAX_VALUE);
+    GemFireChunk chunk4 = createValueAsSerializedStoredObject(Long.MAX_VALUE);
+
+    int newSizeForChunk3 = 2;
+    int newSizeForChunk4 = 3;
+
+    assertThat(chunk3.compareTo(chunk4)).isEqualTo(Integer.signum(newSizeForChunk3 - newSizeForChunk4));
+
+    chunk1.release();
+    chunk4.release();
+  }
+
+  @Test
+  public void setSerializedShouldSetTheSerializedBit() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    int headerBeforeSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+
+    assertThat(chunk.isSerialized()).isFalse();
+
+    chunk.setSerialized(true); // set to true
+
+    assertThat(chunk.isSerialized()).isTrue();
+
+    int headerAfterSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+
+    assertThat(headerAfterSerializedBitSet).isEqualTo(headerBeforeSerializedBitSet | 0x80000000/* IS_SERIALIZED_BIT */);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void setSerialziedShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.setSerialized(true);
+
+    chunk.release();
+  }
+
+  @Test
+  public void setCompressedShouldSetTheCompressedBit() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    int headerBeforeCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+
+    assertThat(chunk.isCompressed()).isFalse();
+
+    chunk.setCompressed(true); // set to true
+
+    assertThat(chunk.isCompressed()).isTrue();
+
+    int headerAfterCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+
+    assertThat(headerAfterCompressedBitSet).isEqualTo(headerBeforeCompressedBitSet | 0x40000000/* IS_SERIALIZED_BIT */);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void setCompressedShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.setCompressed(true);
+
+    chunk.release();
+  }
+
+  @Test
+  public void setDataSizeShouldSetTheDataSizeBits() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    int beforeSize = chunk.getDataSize();
+
+    chunk.setDataSize(2);
+
+    int afterSize = chunk.getDataSize();
+
+    assertThat(afterSize).isEqualTo(2);
+    assertThat(afterSize).isNotEqualTo(beforeSize);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void setDataSizeShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.setDataSize(1);
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void initializeUseCountShouldThrowIllegalStateExceptionIfChunkIsAlreadyRetained() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.retain();
+    chunk.initializeUseCount();
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void initializeUseCountShouldThrowIllegalStateExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.initializeUseCount();
+
+    chunk.release();
+  }
+
+  @Test
+  public void isSerializedPdxInstanceShouldReturnTrueIfItsPDXInstance() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    byte[] serailizedValue = chunk.getSerializedValue();
+    serailizedValue[0] = DSCODE.PDX;
+    chunk.setSerializedValue(serailizedValue);
+
+    assertThat(chunk.isSerializedPdxInstance()).isTrue();
+
+    serailizedValue = chunk.getSerializedValue();
+    serailizedValue[0] = DSCODE.PDX_ENUM;
+    chunk.setSerializedValue(serailizedValue);
+
+    assertThat(chunk.isSerializedPdxInstance()).isTrue();
+
+    serailizedValue = chunk.getSerializedValue();
+    serailizedValue[0] = DSCODE.PDX_INLINE_ENUM;
+    chunk.setSerializedValue(serailizedValue);
+
+    assertThat(chunk.isSerializedPdxInstance()).isTrue();
+
+    chunk.release();
+  }
+
+  @Test
+  public void isSerializedPdxInstanceShouldReturnFalseIfItsNotPDXInstance() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.isSerializedPdxInstance()).isFalse();
+
+    chunk.release();
+  }
+
+  @Test
+  public void checkDataEqualsByChunk() {
+    GemFireChunk chunk1 = createValueAsSerializedStoredObject(getValue());
+    GemFireChunk sameAsChunk1 = chunk1;
+
+    assertThat(chunk1.checkDataEquals(sameAsChunk1)).isTrue();
+
+    GemFireChunk unserializedChunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk1.checkDataEquals(unserializedChunk)).isFalse();
+
+    GemFireChunk chunkDifferBySize = createValueAsSerializedStoredObject(getValue());
+    chunkDifferBySize.setSize(0);
+    assertThat(chunk1.checkDataEquals(chunkDifferBySize)).isFalse();
+
+    GemFireChunk chunkDifferByValue = createValueAsSerializedStoredObject(Long.MAX_VALUE - 1);
+    assertThat(chunk1.checkDataEquals(chunkDifferByValue)).isFalse();
+
+    GemFireChunk newChunk1 = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk1.checkDataEquals(newChunk1)).isTrue();
+
+    chunk1.release();
+    unserializedChunk.release();
+    chunkDifferBySize.release();
+    chunkDifferByValue.release();
+    newChunk1.release();
+  }
+
+  @Test
+  public void checkDataEqualsBySerializedValue() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.checkDataEquals(new byte[1])).isFalse();
+
+    GemFireChunk chunkDifferByValue = createValueAsSerializedStoredObject(Long.MAX_VALUE - 1);
+    assertThat(chunk.checkDataEquals(chunkDifferByValue.getSerializedValue())).isFalse();
+
+    GemFireChunk newChunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.checkDataEquals(newChunk.getSerializedValue())).isTrue();
+
+    chunk.release();
+    chunkDifferByValue.release();
+    newChunk.release();
+  }
+
+  @Test
+  public void getDecompressedBytesShouldReturnDecompressedBytesIfCompressed() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = true;
+    boolean isCompressed = true;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    RegionEntryContext regionContext = mock(RegionEntryContext.class);
+    CachePerfStats cacheStats = mock(CachePerfStats.class);
+    Compressor compressor = mock(Compressor.class);
+
+    long startTime = 10000L;
+
+    // mock required things
+    when(regionContext.getCompressor()).thenReturn(compressor);
+    when(compressor.decompress(regionEntryValueAsBytes)).thenReturn(regionEntryValueAsBytes);
+    when(regionContext.getCachePerfStats()).thenReturn(cacheStats);
+    when(cacheStats.startDecompression()).thenReturn(startTime);
+
+    // invoke the thing
+    byte[] bytes = chunk.getDecompressedBytes(regionContext);
+
+    // verify the thing happened
+    verify(cacheStats, atLeastOnce()).startDecompression();
+    verify(compressor, times(1)).decompress(regionEntryValueAsBytes);
+    verify(cacheStats, atLeastOnce()).endDecompression(startTime);
+
+    assertArrayEquals(regionEntryValueAsBytes, bytes);
+
+    chunk.release();
+  }
+
+  @Test
+  public void incSizeShouldIncrementSize() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    int beforeSize = chunk.getSize();
+
+    chunk.incSize(1);
+    assertThat(chunk.getSize()).isEqualTo(beforeSize + 1);
+
+    chunk.incSize(2);
+    assertThat(chunk.getSize()).isEqualTo(beforeSize + 1 + 2);
+
+    chunk.release();
+  }
+
+  @Test
+  public void readyForFreeShouldResetTheRefCount() {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    int refCountBeforeFreeing = chunk.getRefCount();
+    assertThat(refCountBeforeFreeing).isEqualTo(1);
+
+    chunk.readyForFree();
+
+    int refCountAfterFreeing = chunk.getRefCount();
+    assertThat(refCountAfterFreeing).isEqualTo(0);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void readyForAllocationShouldThrowExceptionIfAlreadyAllocated() {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    // chunk is already allocated when we created it, so calling readyForAllocation should throw exception.
+    chunk.readyForAllocation(GemFireChunk.TYPE);
+
+    chunk.release();
+  }
+
+  @Test
+  public void checkIsAllocatedShouldReturnIfAllocated() {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.checkIsAllocated();
+
+    chunk.release();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void checkIsAllocatedShouldThrowExceptionIfNotAllocated() {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.release();
+    chunk.checkIsAllocated();
+
+    chunk.release();
+  }
+
+  @Test
+  public void sendToShouldWriteSerializedValueToDataOutputIfValueIsSerialized() throws IOException {
+    Chunk chunk = createValueAsSerializedStoredObject(getValue());
+    Chunk spyChunk = spy(chunk);
+
+    HeapDataOutputStream dataOutput = mock(HeapDataOutputStream.class);
+    ByteBuffer directByteBuffer = ByteBuffer.allocate(1024);
+
+    doReturn(directByteBuffer).when(spyChunk).createDirectByteBuffer();
+    doNothing().when(dataOutput).write(directByteBuffer);
+
+    spyChunk.sendTo(dataOutput);
+
+    verify(dataOutput, times(1)).write(directByteBuffer);
+
+    chunk.release();
+  }
+
+  @Test
+  public void sendToShouldWriteUnserializedValueToDataOutputIfValueIsUnserialized() throws IOException {
+    byte[] regionEntryValue = getValueAsByteArray();
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    // writeByte is a final method and cannot be mocked, so creating a real one
+    HeapDataOutputStream dataOutput = new HeapDataOutputStream(Version.CURRENT);
+
+    chunk.sendTo(dataOutput);
+
+    byte[] actual = dataOutput.toByteArray();
+
+    byte[] expected = new byte[regionEntryValue.length + 2];
+    expected[0] = DSCODE.BYTE_ARRAY;
+    expected[1] = (byte) regionEntryValue.length;
+    System.arraycopy(regionEntryValue, 0, expected, 2, regionEntryValue.length);
+
+    assertNotNull(dataOutput);
+    assertThat(actual).isEqualTo(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void sendAsByteArrayShouldWriteValueToDataOutput() throws IOException {
+    byte[] regionEntryValue = getValueAsByteArray();
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    // writeByte is a final method and cannot be mocked, so creating a real one
+    HeapDataOutputStream dataOutput = new HeapDataOutputStream(Version.CURRENT);
+
+    chunk.sendAsByteArray(dataOutput);
+
+    byte[] actual = dataOutput.toByteArray();
+
+    byte[] expected = new byte[regionEntryValue.length + 1];
+    expected[0] = (byte) regionEntryValue.length;
+    System.arraycopy(regionEntryValue, 0, expected, 1, regionEntryValue.length);
+
+    assertNotNull(dataOutput);
+    assertThat(actual).isEqualTo(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void createDirectByteBufferShouldCreateAByteBuffer() {
+    byte[] regionEntryValue = getValueAsByteArray();
+
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    ByteBuffer buffer = chunk.createDirectByteBuffer();
+
+    byte[] actual = new byte[regionEntryValue.length];
+    buffer.get(actual);
+
+    assertArrayEquals(regionEntryValue, actual);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getDirectByteBufferShouldCreateAByteBuffer() {
+    byte[] regionEntryValue = getValueAsByteArray();
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(regionEntryValue);
+
+    ByteBuffer buffer = chunk.createDirectByteBuffer();
+    long bufferAddress = Chunk.getDirectByteBufferAddress(buffer);
+
+    // returned address should be starting of the value (after skipping HEADER_SIZE bytes)
+    assertEquals(chunk.getMemoryAddress() + Chunk.OFF_HEAP_HEADER_SIZE, bufferAddress);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getAddressForReadingShouldFailIfItsOutsideOfChunk() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getAddressForReading(0, chunk.getDataSize() + 1);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getAddressForReadingShouldReturnDataAddressFromGivenOffset() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    int offset = 1;
+    long requestedAddress = chunk.getAddressForReading(offset, 1);
+
+    assertThat(requestedAddress).isEqualTo(chunk.getBaseDataAddress() + offset);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSizeInBytesShouldReturnSize() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    assertThat(chunk.getSizeInBytes()).isEqualTo(chunk.getSize());
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getUnsafeAddressShouldFailIfOffsetIsNegative() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getUnsafeAddress(-1, 1);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getUnsafeAddressShouldFailIfSizeIsNegative() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getUnsafeAddress(1, -1);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void getUnsafeAddressShouldFailIfItsOutsideOfChunk() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+    chunk.getUnsafeAddress(0, chunk.getDataSize() + 1);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getUnsafeAddressShouldReturnUnsafeAddress() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    int offset = 1;
+    long unsafeAddress = chunk.getUnsafeAddress(offset, 1);
+
+    assertThat(unsafeAddress).isEqualTo(chunk.getBaseDataAddress() + offset);
+
+    chunk.release();
+  }
+
+  @Test(expected = AssertionError.class)
+  public void readByteAndWriteByteShouldFailIfOffsetIsOutside() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    chunk.readByte(chunk.getDataSize() + 1);
+
+    chunk.writeByte(chunk.getDataSize() + 1, Byte.MAX_VALUE);
+
+    chunk.release();
+  }
+
+  @Test
+  public void writeByteShouldWriteAtCorrectLocation() {
+    GemFireChunk chunk = createValueAsSerializedStoredObject(getValue());
+
+    byte valueBeforeWrite = chunk.readByte(2);
+
+    Byte expected = Byte.MAX_VALUE;
+    chunk.writeByte(2, expected);
+
+    Byte actual = chunk.readByte(2);
+
+    assertThat(actual).isNotEqualTo(valueBeforeWrite);
+    assertThat(actual).isEqualTo(expected);
+
+    chunk.release();
+  }
+
+  @Test
+  public void retainShouldIncrementRefCount() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.retain();
+    assertThat(chunk.getRefCount()).isEqualTo(2);
+
+    chunk.retain();
+    assertThat(chunk.getRefCount()).isEqualTo(3);
+
+    chunk.release();
+    chunk.release();
+    chunk.release();
+    boolean retainAfterRelease = chunk.retain();
+
+    assertThat(retainAfterRelease).isFalse();
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void retainShouldThrowExceptionAfterMaxNumberOfTimesRetained() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    // max retain Chunk.MAX_REF_COUNT
+    int MAX_REF_COUNT = 0xFFFF;
+
+    // loop though and invoke retain for MAX_REF_COUNT-1 times, as create chunk above counted as one reference
+    for (int i = 0; i < MAX_REF_COUNT - 1; i++)
+      chunk.retain();
+
+    // invoke for the one more time should throw exception
+    chunk.retain();
+  }
+
+  @Test
+  public void releaseShouldDecrementRefCount() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.retain();
+    chunk.retain();
+    assertThat(chunk.getRefCount()).isEqualTo(3);
+
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(2);
+
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.retain();
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(1);
+
+    chunk.release();
+    assertThat(chunk.getRefCount()).isEqualTo(0);
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void releaseShouldThrowExceptionIfChunkIsAlreadyReleased() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    chunk.release();
+  }
+
+  @Test
+  public void testToStringForOffHeapByteSource() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    String expected = ":<dataSize=" + chunk.getDataSize() + " refCount=" + chunk.getRefCount() + " addr=" + Long.toHexString(chunk.getMemoryAddress()) + ">";
+    assertThat(chunk.toStringForOffHeapByteSource()).endsWith(expected);
+
+    // test toString
+    Chunk spy = spy(chunk);
+    spy.toString();
+    verify(spy, times(1)).toStringForOffHeapByteSource();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getStateShouldReturnAllocatedIfRefCountIsGreaterThanZero() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertEquals(State.ALLOCATED, chunk.getState());
+
+    chunk.release();
+  }
+
+  @Test
+  public void getStateShouldReturnDeallocatedIfRefCountIsZero() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.release();
+    assertEquals(State.DEALLOCATED, chunk.getState());
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void getNextBlockShouldThrowUnSupportedOperationException() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.getNextBlock();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getBlockSizeShouldBeSameSameGetSize() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertEquals(chunk.getSize(), chunk.getBlockSize());
+
+    chunk.release();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void copyBytesShouldThrowUnSupportedOperationException() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.copyBytes(1, 2, 1);
+
+    chunk.release();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void getSlabIdShouldThrowUnSupportedOperationException() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    chunk.getSlabId();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getFreeListIdShouldReturnMinusOne() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getFreeListId()).isEqualTo(-1);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getDataTypeShouldReturnNull() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getDataType()).isNull();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getDataDataShouldReturnNull() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    assertThat(chunk.getDataValue()).isNull();
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void getRawBytesShouldThrowExceptionIfValueIsCompressed() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = true;
+    boolean isCompressed = true;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    chunk.getRawBytes();
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSerializedValueShouldSerializeTheValue() {
+    Object regionEntryValue = getValue();
+    byte[] regionEntryValueAsBytes = convertValueToByteArray(regionEntryValue);
+
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    byte[] serializedValue = chunk.getSerializedValue();
+
+    assertThat(serializedValue).isEqualTo(EntryEventImpl.serialize(regionEntryValueAsBytes));
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSrcTypeOrdinalFromAddressShouldReturnOrdinal() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    assertThat(Chunk.getSrcTypeOrdinal(chunk.getMemoryAddress())).isEqualTo(4);
+
+    chunk.release();
+  }
+
+  @Test
+  public void getSrcTypeOrdinalFromRawBitsShouldReturnOrdinal() {
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+
+    int rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4 /* REF_COUNT_OFFSET */);
+    assertThat(Chunk.getSrcTypeOrdinalFromRawBits(rawBits)).isEqualTo(4);
+
+    chunk.release();
+  }
+
+  @Test
+  public void fillShouldFillTheChunk() {
+    boolean isSerialized = false;
+    boolean isCompressed = false;
+
+    GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(new byte[100], isSerialized, isCompressed, GemFireChunk.TYPE);
+
+    // first fill the unused part with FILL_PATTERN
+    Chunk.fill(chunk.getMemoryAddress());
+
+    // Validate that it is filled
+    chunk.validateFill();
+
+    chunk.release();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSliceJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSliceJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSliceJUnitTest.java
new file mode 100644
index 0000000..39ee620
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkSliceJUnitTest.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.offheap;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GemFireChunkSliceJUnitTest extends GemFireChunkJUnitTest {
+
+  @Test
+  public void sliceShouldHaveAValidDataSize() {
+    int position = 1;
+    int end = 2;
+
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    GemFireChunkSlice slice = (GemFireChunkSlice) chunk.slice(position, end);
+
+    assertNotNull(slice);
+    assertEquals(GemFireChunkSlice.class, slice.getClass());
+
+    assertEquals(end - position, slice.getDataSize());
+  }
+
+  @Test
+  public void sliceShouldHaveAValidBaseDataAddress() {
+    int position = 1;
+    int end = 2;
+
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    GemFireChunkSlice slice = (GemFireChunkSlice) chunk.slice(position, end);
+
+    assertNotNull(slice);
+    assertEquals(GemFireChunkSlice.class, slice.getClass());
+
+    assertEquals(chunk.getBaseDataAddress() + position, slice.getBaseDataAddress());
+  }
+
+  @Test
+  public void sliceShouldHaveAValidBaseOffset() {
+    int position = 1;
+    int end = 2;
+
+    GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
+    GemFireChunkSlice slice = (GemFireChunkSlice) chunk.slice(position, end);
+
+    assertNotNull(slice);
+    assertEquals(GemFireChunkSlice.class, slice.getClass());
+
+    assertEquals(chunk.getBaseDataOffset() + position, slice.getBaseDataOffset());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fdec70bc/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java
new file mode 100644
index 0000000..d4f9e97
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/StoredObjectTestSuite.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.offheap;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@Suite.SuiteClasses({
+	DataAsAddressJUnitTest.class,
+	GemFireChunkJUnitTest.class,
+	ChunkWithHeapFormJUnitTest.class,
+	GemFireChunkSliceJUnitTest.class,
+	GemFireChunkFactoryJUnitTest.class
+})
+@RunWith(Suite.class)
+public class StoredObjectTestSuite {
+
+}



[35/50] [abbrv] incubator-geode git commit: GEODE-666: Update awaitility dependency from 1.6.5 to 1.7.0

Posted by je...@apache.org.
GEODE-666: Update awaitility dependency from 1.6.5 to 1.7.0


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

Branch: refs/heads/feature/GEODE-14
Commit: c30e533853c25abf23ccee8b5a63255fce4c1885
Parents: ae2d529
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Dec 14 09:31:58 2015 -0800
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Dec 15 14:07:04 2015 -0800

----------------------------------------------------------------------
 gradle/dependency-versions.properties | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c30e5338/gradle/dependency-versions.properties
----------------------------------------------------------------------
diff --git a/gradle/dependency-versions.properties b/gradle/dependency-versions.properties
index 8ea38c3..684e60e 100644
--- a/gradle/dependency-versions.properties
+++ b/gradle/dependency-versions.properties
@@ -22,7 +22,7 @@ annotations.version = 3.0.0
 antlr.version = 2.7.7
 asm.version = 5.0.3
 assertj-core.version = 3.2.0
-awaitility.version = 1.6.5
+awaitility.version = 1.7.0
 bcel.version = 5.2
 catch-exception.version = 1.4.4
 catch-throwable.version = 1.4.4


[42/50] [abbrv] incubator-geode git commit: Modifying the fix for handling old client IDs

Posted by je...@apache.org.
Modifying the fix for handling old client IDs

The previous fix for handling old client identifiers left the IDs always
thinking that the client was running version GFE_82.  The new fix makes
the client defer to its encapsulated InternalDistributedMember identifier
to determine the client's version and handles deserialization of the
encapsulated pre-GFE_90 identifier in InternalDistributedMember.fromData().

This fix has the added benefit of reducing the size of client identifiers.


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

Branch: refs/heads/feature/GEODE-14
Commit: f2a852d880d166d8b4a4a11218a38f49e6016864
Parents: fdec70b
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Thu Dec 17 08:47:10 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Thu Dec 17 08:51:31 2015 -0800

----------------------------------------------------------------------
 .../internal/direct/DirectChannel.java          |  2 +-
 .../membership/InternalDistributedMember.java   | 13 ++++-
 .../internal/membership/NetMember.java          |  4 ++
 .../internal/membership/gms/GMSMember.java      |  7 +--
 .../tier/sockets/ClientProxyMembershipID.java   | 15 ++----
 .../internal/cache/tier/sockets/HandShake.java  | 16 +++----
 .../client/ClientCacheFactoryJUnitTest.java     | 50 ++++++++++++++++++++
 .../sanctionedDataSerializables.txt             | 12 ++---
 8 files changed, 89 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
index d4df3bf..a4245a0 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
@@ -596,7 +596,7 @@ public class DirectChannel {
           logger.trace(LogMarker.DM, "Not a member: {}", destination);
         }
         if (ce == null) ce = new ConnectExceptions();
-        ce.addFailure(destination, new ShunnedMemberException(LocalizedStrings.DirectChannel_SHUNNING_0.toLocalizedString()));
+        ce.addFailure(destination, new ShunnedMemberException(LocalizedStrings.DirectChannel_SHUNNING_0.toLocalizedString(destination)));
       }
       else {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
index b112b92..a77c211 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember.java
@@ -18,6 +18,7 @@ package com.gemstone.gemfire.distributed.internal.membership;
 
 import java.io.DataInput;
 import java.io.DataOutput;
+import java.io.EOFException;
 import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
@@ -1023,7 +1024,11 @@ public final class InternalDistributedMember
     // just in case this is just a non-versioned read
     // from a file we ought to check the version
     if (this.version >= Version.GFE_90.ordinal()) {
-      netMbr.readAdditionalData(in);
+      try {
+        netMbr.readAdditionalData(in);
+      } catch (EOFException e) {
+        // nope - it's from a pre-GEODE client or WAN site
+      }
     }
   }
   
@@ -1254,6 +1259,12 @@ public final class InternalDistributedMember
       sb.append(hostname);
     }
   }*/
+  
+  public final void setVersionObjectForTest(Version v) {
+    this.version = v.ordinal();
+    this.versionObj = v;
+    netMbr.setVersion(v);
+  }
 
   public final Version getVersionObject() {
     return this.versionObj;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java
index 6811073..128f96c 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/NetMember.java
@@ -21,6 +21,8 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.net.InetAddress;
 
+import com.gemstone.gemfire.internal.Version;
+
 /**
  * This is the SPI for the basic element of membership provided in the
  * GemFire system.
@@ -68,6 +70,8 @@ public interface NetMember extends Comparable<NetMember>
   public void setPreferredForCoordinator(boolean preferred);
   
   public byte getMemberWeight();
+  
+  public void setVersion(Version v);
 
   /** write identity information not known by DistributedMember instances */
   public void writeAdditionalData(DataOutput out) throws IOException;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
index b1a4883..84ace6c 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/GMSMember.java
@@ -26,11 +26,9 @@ import org.jgroups.util.UUID;
 
 import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.distributed.DurableClientAttributes;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.membership.MemberAttributes;
 import com.gemstone.gemfire.distributed.internal.membership.NetMember;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
@@ -349,6 +347,10 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
   public void setVmKind(int vmKind) {
     this.vmKind = vmKind;
   }
+  
+  public void setVersion(Version v) {
+    this.versionOrdinal = v.ordinal();
+  }
 
   public void setBirthViewId(int birthViewId) {
     this.vmViewId = birthViewId;
@@ -441,7 +443,6 @@ public class GMSMember implements NetMember, DataSerializableFixedID {
 
   @Override
   public void writeAdditionalData(DataOutput out) throws IOException {
-    // note: member weight and other GMS-only info is not included here
     out.writeLong(uuidMSBs);
     out.writeLong(uuidLSBs);
     out.write(memberWeight);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
index daa07f4..44c6152 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientProxyMembershipID.java
@@ -39,6 +39,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.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.VersionedDataInputStream;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -101,10 +102,6 @@ public final class ClientProxyMembershipID
 
   protected int uniqueId;
   
-  //Version information is not available during the handshake
-  //see comments in HandShake.write()
-  private transient static final Version clientVersion = Version.GFE_82;
-
   // private final String proxyIDStr;
   // private final String clientIdStr ;
 
@@ -353,14 +350,13 @@ public final class ClientProxyMembershipID
 
   public void fromData(DataInput in) throws IOException, ClassNotFoundException
   {
-    
     this.identity = DataSerializer.readByteArray(in);
     this.uniqueId = in.readInt();
 //    {toString(); this.transientPort = ((InternalDistributedMember)this.memberId).getPort();}
   }
   
   public Version getClientVersion() {
-    return this.clientVersion;
+    return ((InternalDistributedMember)getDistributedMember()).getVersionObject();
   }
 
   public String getDSMembership()
@@ -405,15 +401,12 @@ public final class ClientProxyMembershipID
   public DistributedMember getDistributedMember()  {
     if (memberId == null) {      
       ByteArrayInputStream bais = new ByteArrayInputStream(identity);
-      DataInputStream dis = new VersionedDataInputStream(bais, clientVersion);
+      DataInputStream dis = new VersionedDataInputStream(bais, Version.CURRENT);
       try {
         memberId = (DistributedMember)DataSerializer.readObject(dis);
       }
       catch (Exception e) {
-        DistributedSystem ds = InternalDistributedSystem.getAnyInstance();
-        if(ds != null){
-          logger.error(LocalizedMessage.create(LocalizedStrings.ClientProxyMembershipID_UNABLE_TO_DESERIALIZE_MEMBERSHIP_ID), e);
-        }
+        logger.error(LocalizedMessage.create(LocalizedStrings.ClientProxyMembershipID_UNABLE_TO_DESERIALIZE_MEMBERSHIP_ID), e);
       }
     }
     return memberId;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
index be3a7bf..100549a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/HandShake.java
@@ -1314,14 +1314,14 @@ public class HandShake implements ClientHandShake
       //Successful handshake for GATEWAY_TO_GATEWAY mode sets the peer version in connection
       if(communicationMode == Acceptor.GATEWAY_TO_GATEWAY  && !
           (acceptanceCode == REPLY_EXCEPTION_AUTHENTICATION_REQUIRED ||
-              acceptanceCode ==  REPLY_EXCEPTION_AUTHENTICATION_FAILED)) {
-         short wanSiteVersion = Version.readOrdinal(dis);
-         conn.setWanSiteVersion(wanSiteVersion);
-         // establish a versioned stream for the other site, if necessary         
-         if (wanSiteVersion < Version.CURRENT_ORDINAL) {
-           dis = new VersionedDataInputStream(dis, Version.fromOrdinalOrCurrent(wanSiteVersion));
-         }
-        } 
+          acceptanceCode ==  REPLY_EXCEPTION_AUTHENTICATION_FAILED)) {
+        short wanSiteVersion = Version.readOrdinal(dis);
+        conn.setWanSiteVersion(wanSiteVersion);
+        // establish a versioned stream for the other site, if necessary         
+        if (wanSiteVersion < Version.CURRENT_ORDINAL) {
+          dis = new VersionedDataInputStream(dis, Version.fromOrdinalOrCurrent(wanSiteVersion));
+        }
+      } 
 
       // No need to check for return value since DataInputStream already throws
       // EOFException in case of EOF

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java
index d15863b..6c19018 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/client/ClientCacheFactoryJUnitTest.java
@@ -20,6 +20,8 @@ package com.gemstone.gemfire.cache.client;
 import static org.junit.Assert.*;
 import static org.junit.runners.MethodSorters.*;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
 import java.io.File;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -28,20 +30,28 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Properties;
 
+import org.jgroups.util.UUID;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.FixMethodOrder;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.gemstone.gemfire.DataSerializer;
 import com.gemstone.gemfire.cache.RegionService;
 import com.gemstone.gemfire.cache.client.internal.ProxyCache;
 import com.gemstone.gemfire.cache.client.internal.UserAttributes;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
+import com.gemstone.gemfire.distributed.internal.membership.gms.GMSMember;
 import com.gemstone.gemfire.internal.FileUtil;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.VersionedDataInputStream;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.pdx.ReflectionBasedAutoSerializer;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
@@ -290,4 +300,44 @@ public class ClientCacheFactoryJUnitTest {
     .setPdxSerializer(new ReflectionBasedAutoSerializer())
     .create();
   }
+  
+  @Test
+  public void testOldClientIDDeserialization() throws Exception {
+    // during a HandShake a clientID is read w/o knowing the client's
+    // version
+    cc = new ClientCacheFactory().create();
+    GemFireCacheImpl gfc = (GemFireCacheImpl)cc;
+    InternalDistributedMember memberID = (InternalDistributedMember)cc.getDistributedSystem().getDistributedMember();
+    GMSMember gmsID = (GMSMember)memberID.getNetMember();
+    memberID.setVersionObjectForTest(Version.GFE_82);
+    assertEquals(Version.GFE_82, memberID.getVersionObject());
+    ClientProxyMembershipID clientID = ClientProxyMembershipID.getClientId(memberID);
+    HeapDataOutputStream out = new HeapDataOutputStream(Version.GFE_82);
+    DataSerializer.writeObject(clientID, out);
+
+    DataInputStream in = new VersionedDataInputStream(new ByteArrayInputStream(out.toByteArray()), Version.CURRENT); 
+    ClientProxyMembershipID newID = DataSerializer.readObject(in);
+    InternalDistributedMember newMemberID = (InternalDistributedMember)newID.getDistributedMember();
+    assertEquals(Version.GFE_82, newMemberID.getVersionObject());
+    assertEquals(Version.GFE_82, newID.getClientVersion());
+    GMSMember newGmsID = (GMSMember)newMemberID.getNetMember();
+    assertEquals(0, newGmsID.getUuidLSBs());
+    assertEquals(0, newGmsID.getUuidMSBs());
+    
+    gmsID.setUUID(new UUID(1234l, 5678l));
+    memberID.setVersionObjectForTest(Version.CURRENT);
+    clientID = ClientProxyMembershipID.getClientId(memberID);
+    out = new HeapDataOutputStream(Version.CURRENT);
+    DataSerializer.writeObject(clientID, out);
+
+    in = new VersionedDataInputStream(new ByteArrayInputStream(out.toByteArray()), Version.CURRENT);
+    newID = DataSerializer.readObject(in);
+    newMemberID = (InternalDistributedMember)newID.getDistributedMember();
+    assertEquals(Version.CURRENT, newMemberID.getVersionObject());
+    assertEquals(Version.CURRENT, newID.getClientVersion());
+    newGmsID = (GMSMember)newMemberID.getNetMember();
+    assertEquals(gmsID.getUuidLSBs(), newGmsID.getUuidLSBs());
+    assertEquals(gmsID.getUuidMSBs(), newGmsID.getUuidMSBs());
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2a852d8/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 d553ad9..d3a141a 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
@@ -329,12 +329,12 @@ fromData,16,2a2bb700172a2bb900180100b50008b1
 toData,16,2a2bb700192b2ab40008b9001a0200b1
 
 com/gemstone/gemfire/distributed/internal/membership/InternalDistributedMember,6
-fromData,29,2a2bb600992ab4001ab2008fb6006fa1000d2ab400282bb900900200b1
-fromDataPre_GFE_7_1_0_0,292,2bb8009a4d2bb9009b01003e2a2bb80087b500062ab2000399000e2c2ab40006b8009ca700072cb60007b500062bb9009d010036041504047e99000704a700040336051504057e99000704a700040336062a1504077e99000704a7000403b500382a2bb9009b0100b500102a2bb9009b0100b500022a2bb9009d0100b500122a2bb80089b5001f2a2bb80087b500172ab40012100da0000e2a2bb80087b50018a700172bb800873a071907c6000c2a1907b8009eb500132bb800873a072bb8008ab6008b36082abb008c5919071508b7008db5001dbb0032592ab400102ab400022ab400122ab400132ab400172ab4001f2ab4001db700333a092a2c1d150515062bb8009fb6006f1909b80027b500282ab7000b2ab400129e000704a7000403b80041b1
-fromDataPre_GFE_9_0_0_0,296,2bb8009a4d2bb9009b01003e2a2bb80087b500062ab2000399000e2c2ab40006b8009ca700072cb60007b500062bb9009d010036041504047e99000704a700040336051504057e99000704a700040336062a1504077e99000704a7000403b500382a2bb9009b0100b500102a2bb9009b0100b500022a2bb9009d0100b500122a2bb80089b5001f2a2bb80087b500172ab40012100da0000e2a2bb80087b50018a700172bb800873a071907c6000c2a1907b8009eb500132bb800873a072bb8008ab6008b36082abb008c5919071508b7008db5001d2a15042bb7008ebb0032592ab400102ab400022ab400122ab400132ab400172ab4001f2ab4001db700333a092a2c1d150515062ab4001a1909b80027b500282ab7000b2ab400129e000704a7000403b80041b1
-toData,29,2a2bb600932ab4001ab2008fb6006fa1000d2ab600942bb900830200b1
-toDataPre_GFE_7_1_0_0,226,2ab400129e000704a7000403b800412ab6004d2bb800952b2ab6004cb9009602002ab400062bb8007a033d2ab40028b9007b01009900071c04803d2ab40028b9006a01009900071c05803d2ab400389900071c07803d2b1c1100ff7e91b9009702002b2ab40010b9009602002b2ab40002b9009602002b2ab40012b9009702002ab4001f2bb8007d2ab400172bb8007a2ab40012100da0000e2ab400182bb8007aa7000e2ab40013b800982bb8007a2ab4001dc700081245a7000a2ab4001db6007e2bb8007a2ab4001dc7000911012ca7000a2ab4001db6007fb800802bb80081b1
-toDataPre_GFE_9_0_0_0,240,2ab400129e000704a7000403b800412ab6004d2bb800952b2ab6004cb9009602002ab400062bb8007a033d2ab40028b9007b01009900071c04803d2ab40028b9006a01009900071c05803d2ab400389900071c07803d1c1008803d2b1c1100ff7e91b9009702002b2ab40010b9009602002b2ab40002b9009602002b2ab40012b9009702002ab4001f2bb8007d2ab400172bb8007a2ab40012100da0000e2ab400182bb8007aa7000e2ab40013b800982bb8007a2ab4001dc700081245a7000a2ab4001db6007e2bb8007a2ab4001dc7000911012ca7000a2ab4001db6007fb800802bb800812b2ab4001a04b80082b1
+fromData,33,2a2bb6009a2ab4001ab20090b60070a100112ab400282bb900910200a700044db1
+fromDataPre_GFE_7_1_0_0,292,2bb8009b4d2bb9009c01003e2a2bb80088b500062ab2000399000e2c2ab40006b8009da700072cb60007b500062bb9009e010036041504047e99000704a700040336051504057e99000704a700040336062a1504077e99000704a7000403b500392a2bb9009c0100b500102a2bb9009c0100b500022a2bb9009e0100b500122a2bb8008ab5001f2a2bb80088b500172ab40012100da0000e2a2bb80088b50018a700172bb800883a071907c6000c2a1907b8009fb500132bb800883a072bb8008bb6008c36082abb008d5919071508b7008eb5001dbb0033592ab400102ab400022ab400122ab400132ab400172ab4001f2ab4001db700343a092a2c1d150515062bb800a0b600701909b80027b500282ab7000b2ab400129e000704a7000403b80042b1
+fromDataPre_GFE_9_0_0_0,296,2bb8009b4d2bb9009c01003e2a2bb80088b500062ab2000399000e2c2ab40006b8009da700072cb60007b500062bb9009e010036041504047e99000704a700040336051504057e99000704a700040336062a1504077e99000704a7000403b500392a2bb9009c0100b500102a2bb9009c0100b500022a2bb9009e0100b500122a2bb8008ab5001f2a2bb80088b500172ab40012100da0000e2a2bb80088b50018a700172bb800883a071907c6000c2a1907b8009fb500132bb800883a072bb8008bb6008c36082abb008d5919071508b7008eb5001d2a15042bb7008fbb0033592ab400102ab400022ab400122ab400132ab400172ab4001f2ab4001db700343a092a2c1d150515062ab4001a1909b80027b500282ab7000b2ab400129e000704a7000403b80042b1
+toData,29,2a2bb600942ab4001ab20090b60070a1000d2ab600952bb900840200b1
+toDataPre_GFE_7_1_0_0,226,2ab400129e000704a7000403b800422ab6004e2bb800962b2ab6004db9009702002ab400062bb8007b033d2ab40028b9007c01009900071c04803d2ab40028b9006b01009900071c05803d2ab400399900071c07803d2b1c1100ff7e91b9009802002b2ab40010b9009702002b2ab40002b9009702002b2ab40012b9009802002ab4001f2bb8007e2ab400172bb8007b2ab40012100da0000e2ab400182bb8007ba7000e2ab40013b800992bb8007b2ab4001dc700081246a7000a2ab4001db6007f2bb8007b2ab4001dc7000911012ca7000a2ab4001db60080b800812bb80082b1
+toDataPre_GFE_9_0_0_0,240,2ab400129e000704a7000403b800422ab6004e2bb800962b2ab6004db9009702002ab400062bb8007b033d2ab40028b9007c01009900071c04803d2ab40028b9006b01009900071c05803d2ab400399900071c07803d1c1008803d2b1c1100ff7e91b9009802002b2ab40010b9009702002b2ab40002b9009702002b2ab40012b9009802002ab4001f2bb8007e2ab400172bb8007b2ab40012100da0000e2ab400182bb8007ba7000e2ab40013b800992bb8007b2ab4001dc700081246a7000a2ab4001db6007f2bb8007b2ab4001dc7000911012ca7000a2ab4001db60080b800812bb800822b2ab4001a04b80083b1
 
 com/gemstone/gemfire/distributed/internal/membership/MemberAttributes,2
 fromData,73,2a2bb900270100b500022a2bb900270100b500032a2bb900270100b500042a2bb80028b500092a2bb80029b500072bb800284d2bb8002ab6002b3e2abb002c592c1db7002db5000ab1


[29/50] [abbrv] incubator-geode git commit: Revert "Revert "Removing TCPConduit's Stub ID class""

Posted by je...@apache.org.
Revert "Revert "Removing TCPConduit's Stub ID class""

This reverts commit 507f2f3a905e70fcabed9b83d4dc966ef3e9e6ec.


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

Branch: refs/heads/feature/GEODE-14
Commit: 4bf4557b2cfe12b9396aeb43dd0b916cbcb98b89
Parents: ec9d16a
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 15 08:21:27 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 15 08:21:27 2015 -0800

----------------------------------------------------------------------
 .../internal/DistributionManager.java           |   8 -
 .../distributed/internal/StartupMessage.java    |   1 -
 .../internal/direct/DirectChannel.java          |  93 +------
 .../internal/direct/MissingStubException.java   |  37 ---
 .../internal/direct/ShunnedMemberException.java |  34 +++
 .../internal/membership/MembershipManager.java  |  29 +-
 .../gms/mgr/GMSMembershipManager.java           | 197 ++-----------
 .../internal/i18n/ParentLocalizedStrings.java   |   6 +-
 .../gemfire/internal/tcp/Connection.java        | 117 ++++----
 .../gemfire/internal/tcp/ConnectionTable.java   |  91 +++---
 .../internal/tcp/MemberShunnedException.java    |   7 +-
 .../gemfire/internal/tcp/ServerDelegate.java    |   5 +-
 .../com/gemstone/gemfire/internal/tcp/Stub.java | 164 -----------
 .../gemfire/internal/tcp/TCPConduit.java        | 274 +++----------------
 .../internal/DistributionManagerDUnitTest.java  |   6 +-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |  31 +--
 .../internal/tcp/ConnectionJUnitTest.java       |   3 +-
 17 files changed, 233 insertions(+), 870 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
index 964845c..e3c342a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
@@ -91,7 +91,6 @@ import com.gemstone.gemfire.internal.sequencelog.MembershipLogger;
 import com.gemstone.gemfire.internal.tcp.Connection;
 import com.gemstone.gemfire.internal.tcp.ConnectionTable;
 import com.gemstone.gemfire.internal.tcp.ReenteredConnectException;
-import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantLock;
 
 /**
@@ -2715,13 +2714,6 @@ public class DistributionManager
       return false; // no peers, we are alone.
     }
 
-    // ensure we have stubs for everyone else
-    Iterator it = allOthers.iterator();
-    while (it.hasNext()) {
-      InternalDistributedMember member = (InternalDistributedMember)it.next();
-      membershipManager.getStubForMember(member);
-    }
-
     try {
       ok = op.sendStartupMessage(allOthers, STARTUP_TIMEOUT, equivs,
           redundancyZone, enforceUniqueZone());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
index 96f8b60..01f8c62 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
@@ -37,7 +37,6 @@ import com.gemstone.gemfire.internal.InternalDataSerializer.SerializerAttributes
 import com.gemstone.gemfire.internal.InternalInstantiator.InstantiatorAttributesHolder;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.tcp.Stub;
 
 /**
  * A message that is sent to all other distribution manager when

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
index 14ff923..d4df3bf 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
@@ -38,6 +38,7 @@ import com.gemstone.gemfire.InternalGemFireException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.ToDataException;
 import com.gemstone.gemfire.cache.TimeoutException;
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
@@ -64,7 +65,6 @@ import com.gemstone.gemfire.internal.tcp.Connection;
 import com.gemstone.gemfire.internal.tcp.ConnectionException;
 import com.gemstone.gemfire.internal.tcp.MemberShunnedException;
 import com.gemstone.gemfire.internal.tcp.MsgStreamer;
-import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.tcp.TCPConduit;
 import com.gemstone.gemfire.internal.util.Breadcrumbs;
 import com.gemstone.gemfire.internal.util.concurrent.ReentrantSemaphore;
@@ -115,13 +115,6 @@ public class DirectChannel {
     }
     
     /**
-     * Returns the endpoint ID for the direct channel
-     */
-    public Stub getLocalStub() {
-      return conduit.getId();
-    }
-    
-    /**
      * when the initial number of members is known, this method is invoked
      * to ensure that connections to those members can be established in a
      * reasonable amount of time.  See bug 39848 
@@ -181,7 +174,7 @@ public class DirectChannel {
         this.groupOrderedSenderSem = new ReentrantSemaphore(MAX_GROUP_SENDERS);
         this.groupUnorderedSenderSem = new ReentrantSemaphore(MAX_GROUP_SENDERS);
         logger.info(LocalizedMessage.create(
-            LocalizedStrings.DirectChannel_GEMFIRE_P2P_LISTENER_STARTED_ON__0, conduit.getId()));
+            LocalizedStrings.DirectChannel_GEMFIRE_P2P_LISTENER_STARTED_ON__0, conduit.getLocalAddr()));
 
       }
       catch (ConnectionException ce) {
@@ -192,48 +185,6 @@ public class DirectChannel {
     }
 
  
-//   /**
-//    * 
-//    * @param addr destination for the message
-//    * @param stubMap map containing all the stubs
-//    * @param msg the original message
-//    * @param msgBuf the serialized message
-//    * @param directAck true if we need an ack
-//    * @param processorType the type (serialized, etc.)
-//    * @return if directAck, the Connection that needs the acknowledgment
-//    * @throws MissingStubException if we do not have a Stub for the recipient
-//    * @throws IOException if the message could not be sent
-//    */
-//   private Connection attemptSingleSend(MembershipManager mgr,
-//       InternalDistributedMember addr,
-//       DistributionMessage msg, ByteBuffer msgBuf,
-//       boolean directAck, int processorType)
-//       throws MissingStubException, IOException
-//   {
-//     if (!msg.deliverToSender() && localAddr.equals(addr))
-//       return null;
-
-//     if (addr == null)
-//       return null;
-//     Stub dest = mgr.getStubForMember(addr);
-//     if (dest == null) {
-//       // This should only happen if the member is no longer in the view.
-//       Assert.assertTrue(!mgr.memberExists(addr));
-//       throw new MissingStubException("No stub");
-//     }
-//     try {
-//       msgBuf.position(0); // fix for bug#30680
-//       Connection con = conduit.sendSync(dest, msgBuf, processorType, msg);
-//       if (directAck)
-//         return con;
-//       else
-//         return null;
-//     }
-//     catch(IOException t) {
-//       throw t;
-//       }
-//   }
-
   /**
    * Return how many concurrent operations should be allowed by default.
    * since 6.6, this has been raised to Integer.MAX value from the number
@@ -639,22 +590,13 @@ public class DirectChannel {
         continue;
       }
 
-      Stub stub = mgr.getStubForMember(destination);
-      if (stub == null) {
+      if (!mgr.memberExists(destination) || mgr.shutdownInProgress() || mgr.isShunned(destination)) {
         // This should only happen if the member is no longer in the view.
         if (logger.isTraceEnabled(LogMarker.DM)) {
-          logger.trace(LogMarker.DM, "No Stub for {}", destination);
+          logger.trace(LogMarker.DM, "Not a member: {}", destination);
         }
-        // The only time getStubForMember returns null is if we are
-        // shunning that member or we are shutting down.
-        // So the following assertion is wrong:
-        //Assert.assertTrue(!mgr.memberExists(destination));
-        // instead we should:
-        // Assert.assertTrue(mgr.shutdownInProgress() || mgr.isShunned(destination));
-        //but this is not worth doing and isShunned is not public.
-        // SO the assert has been deadcoded.
         if (ce == null) ce = new ConnectExceptions();
-        ce.addFailure(destination, new MissingStubException(LocalizedStrings.DirectChannel_NO_STUB_0.toLocalizedString()));
+        ce.addFailure(destination, new ShunnedMemberException(LocalizedStrings.DirectChannel_SHUNNING_0.toLocalizedString()));
       }
       else {
         try {
@@ -662,8 +604,8 @@ public class DirectChannel {
           if (ackTimeout > 0) {
             startTime = System.currentTimeMillis();
           }
-          Connection con = conduit.getConnection(destination, stub,
-              preserveOrder, retry, startTime, ackTimeout, ackSDTimeout);
+          Connection con = conduit.getConnection(destination, preserveOrder,
+              retry, startTime, ackTimeout, ackSDTimeout);
           
           con.setInUse(true, startTime, 0, 0, null); // fix for bug#37657
           cons.add(con);
@@ -823,7 +765,7 @@ public class DirectChannel {
   }
 
   
-  public void receive(DistributionMessage msg, int bytesRead, Stub connId) {
+  public void receive(DistributionMessage msg, int bytesRead) {
     if (disconnected) {
       return;
     }
@@ -844,10 +786,6 @@ public class DirectChannel {
     }
   }
 
-//  public void newMemberConnected(InternalDistributedMember member, Stub id) {
-//    receiver.newMemberConnected(member, id);
-//  }
-
   public InternalDistributedMember getLocalAddress() {
     return this.localAddr;
   }
@@ -930,13 +868,6 @@ public class DirectChannel {
     }
   }
   
-  /** Create a TCPConduit stub from a JGroups InternalDistributedMember */
-  public Stub createConduitStub(InternalDistributedMember addr) {
-    int port = addr.getDirectChannelPort();
-    Stub stub = new Stub(addr.getInetAddress(), port, addr.getVmViewId());
-    return stub;
-  }
-  
   public void closeEndpoint(InternalDistributedMember member, String reason) {
     closeEndpoint(member, reason, true);
   }
@@ -948,7 +879,7 @@ public class DirectChannel {
   public void closeEndpoint(InternalDistributedMember member, String reason, boolean notifyDisconnect) {
     TCPConduit tc = this.conduit;
     if (tc != null) {
-      tc.removeEndpoint(createConduitStub(member), reason, notifyDisconnect);
+      tc.removeEndpoint(member, reason, notifyDisconnect);
     }
   }
 
@@ -962,7 +893,7 @@ public class DirectChannel {
    *    the map to add the state to
    * @since 5.1
    */
-  public void getChannelStates(Stub member, Map result)
+  public void getChannelStates(DistributedMember member, Map result)
   {
     TCPConduit tc = this.conduit;
     if (tc != null) {
@@ -974,7 +905,7 @@ public class DirectChannel {
    * wait for the given connections to process the number of messages
    * associated with the connection in the given map
    */
-  public void waitForChannelState(Stub member, Map channelState)
+  public void waitForChannelState(DistributedMember member, Map channelState)
     throws InterruptedException
   {
     if (Thread.interrupted()) throw new InterruptedException();
@@ -987,7 +918,7 @@ public class DirectChannel {
   /**
    * returns true if there are still receiver threads for the given member
    */
-  public boolean hasReceiversFor(Stub mbr) {
+  public boolean hasReceiversFor(DistributedMember mbr) {
     return this.conduit.hasReceiversFor(mbr);
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java
deleted file mode 100644
index 49b4486..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.distributed.internal.direct;
-
-import com.gemstone.gemfire.GemFireCheckedException;
-
-/**
- * Exception thrown when the TCPConduit is unable to acquire a stub
- * for the given recipient.
- * 
- * @author jpenney
- *
- */
-public class MissingStubException extends GemFireCheckedException
-{
-
-  private static final long serialVersionUID = -6455664684151074915L;
-
-  public MissingStubException(String msg) {
-    super(msg);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java
new file mode 100644
index 0000000..59db762
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed.internal.direct;
+
+import com.gemstone.gemfire.GemFireCheckedException;
+
+/**
+ * Exception thrown when a member is no longer in the distributed system
+ * 
+ */
+public class ShunnedMemberException extends GemFireCheckedException
+{
+
+  private static final long serialVersionUID = -6455664684151074915L;
+
+  public ShunnedMemberException(String msg) {
+    super(msg);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
index a46680b..7416efa 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
@@ -27,7 +27,6 @@ import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
-import com.gemstone.gemfire.internal.tcp.Stub;
 
 /**
  * A MembershipManager is responsible for reporting a MemberView, as well as
@@ -74,7 +73,7 @@ public interface MembershipManager {
    * @param m the member
    * @return true if it still exists
    */
-  public boolean memberExists(InternalDistributedMember m);
+  public boolean memberExists(DistributedMember m);
   
   /**
    * Is this manager still connected?  If it has not been initialized, this
@@ -143,25 +142,6 @@ public interface MembershipManager {
   throws NotSerializableException;
   
   /**
-   * Return a {@link Stub} referring to the given member.  A <em>null</em> may
-   * be returned if the system is not employing stubs for communication.
-   * 
-   * @param m the member
-   * @return the stub
-   */
-  public Stub getStubForMember(InternalDistributedMember m);
-  
-  /**
-   * Return a {@link InternalDistributedMember} associated with the given Stub.  This
-   * method may return a null if Stubs are not being used.
-   * @param s Stub to look up
-   * @param validated true if member must be in the current view
-   * @return the member associated with the given stub, if any
-   */
-  public InternalDistributedMember getMemberForStub(Stub s, boolean validated);
-  
-  
-  /**
    * Indicates to the membership manager that the system is shutting down.
    * Typically speaking, this means that new connection attempts are to be
    * ignored and disconnect failures are to be (more) tolerated.
@@ -286,7 +266,7 @@ public interface MembershipManager {
    */
   public void warnShun(DistributedMember mbr);
   
-  public boolean addSurpriseMember(DistributedMember mbr, Stub stub);
+  public boolean addSurpriseMember(DistributedMember mbr);
   
   /** if a StartupMessage is going to reject a new member, this should be used
    * to make sure we don't keep that member on as a "surprise member"
@@ -307,6 +287,11 @@ public interface MembershipManager {
    * @return true if the member is a surprise member
    */
   public boolean isSurpriseMember(DistributedMember m);
+  
+  /**
+   * Returns true if the member is being shunned
+   */
+  public boolean isShunned(DistributedMember m);
 
   /**
    * Forces use of UDP for communications in the current thread.  UDP is

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index 0b7a544..7be0a3a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -94,7 +94,6 @@ import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.shared.StringPrintWriter;
 import com.gemstone.gemfire.internal.tcp.ConnectExceptions;
 import com.gemstone.gemfire.internal.tcp.MemberShunnedException;
-import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.util.Breadcrumbs;
 
 public class GMSMembershipManager implements MembershipManager, Manager
@@ -156,7 +155,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     boolean crashed;
     String reason;
     DistributionMessage dmsg;
-    Stub stub;
     NetView gmsView;
     
     @Override
@@ -165,7 +163,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       sb.append("kind=");
       switch (kind) {
       case SURPRISE_CONNECT:
-        sb.append("connect; member = <" + member + ">; stub = " + stub);
+        sb.append("connect; member = <" + member + ">");
         break;
       case VIEW:
         String text = gmsView.toString();
@@ -184,12 +182,10 @@ public class GMSMembershipManager implements MembershipManager, Manager
     /**
      * Create a surprise connect event
      * @param member the member connecting
-     * @param id the stub
      */
-    StartupEvent(final InternalDistributedMember member, final Stub id) {
+    StartupEvent(final InternalDistributedMember member) {
       this.kind = SURPRISE_CONNECT;
       this.member = member;
-      this.stub = id;
     }
     /**
      * Indicate if this is a surprise connect event
@@ -282,24 +278,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
   volatile boolean hasJoined;
   
   /**
-   * a map keyed on InternalDistributedMember, values are Stubs that represent direct
-   * channels to other systems
-   * 
-   * Accesses must be under the read or write lock of {@link #latestViewLock}.
-   */
-  protected final Map<InternalDistributedMember, Stub> memberToStubMap = 
-      new ConcurrentHashMap<InternalDistributedMember, Stub>();
-
-  /**
-   * a map of direct channels (Stub) to InternalDistributedMember. key instanceof Stub
-   * value instanceof InternalDistributedMember
-   * 
-   * Accesses must be under the read or write lock of {@link #latestViewLock}.
-   */
-  protected final Map<Stub, InternalDistributedMember> stubToMemberMap = 
-      new ConcurrentHashMap<Stub, InternalDistributedMember>();
-  
-  /**
    * Members of the distributed system that we believe have shut down.
    * Keys are instances of {@link InternalDistributedMember}, values are 
    * Longs indicating the time this member was shunned.
@@ -547,12 +525,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
           }
         }
 
-        // fix for bug #42006, lingering old identity
-        Object oldStub = this.memberToStubMap.remove(m);
-        if (oldStub != null) {
-          this.stubToMemberMap.remove(oldStub);
-        }
-
         if (shutdownInProgress()) {
           addShunnedMember(m);
           continue; // no additions processed after shutdown begins
@@ -806,9 +778,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     
     if (directChannel != null) {
       directChannel.setLocalAddr(address);
-      Stub stub = directChannel.getLocalStub();
-      memberToStubMap.put(address, stub);
-      stubToMemberMap.put(stub, address);
     }
 
     this.hasJoined = true;
@@ -905,17 +874,15 @@ public class GMSMembershipManager implements MembershipManager, Manager
   /**
    * Process a surprise connect event, or place it on the startup queue.
    * @param member the member
-   * @param stub its stub
    */
   protected void handleOrDeferSurpriseConnect(InternalDistributedMember member) {
-    Stub stub = new Stub(member.getInetAddress(), member.getDirectChannelPort(), member.getVmViewId());
     synchronized (startupLock) {
       if (!processingEvents) {
-        startupMessages.add(new StartupEvent(member, stub));
+        startupMessages.add(new StartupEvent(member));
         return;
       }
     }
-    processSurpriseConnect(member, stub);
+    processSurpriseConnect(member);
   }
   
   public void startupMessageFailed(DistributedMember mbr, String failureMessage) {
@@ -941,12 +908,9 @@ public class GMSMembershipManager implements MembershipManager, Manager
    * been added, simply returns; else adds the member.
    * 
    * @param dm the member joining
-   * @param stub the member's stub
    */
-  public boolean addSurpriseMember(DistributedMember dm, 
-      Stub stub) {
+  public boolean addSurpriseMember(DistributedMember dm) {
     final InternalDistributedMember member = (InternalDistributedMember)dm;
-    Stub s = null;
     boolean warn = false;
     
     latestViewLock.writeLock().lock();
@@ -1009,16 +973,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
           startCleanupTimer();
         } // cleanupTimer == null
 
-        // fix for bug #42006, lingering old identity
-        Object oldStub = this.memberToStubMap.remove(member);
-        if (oldStub != null) {
-          this.stubToMemberMap.remove(oldStub);
-        }
-
-        s = stub == null ? getStubForMember(member) : stub;
-        // Make sure that channel information is consistent
-        addChannel(member, s);
-
         // Ensure that the member is accounted for in the view
         // Conjure up a new view including the new member. This is necessary
         // because we are about to tell the listener about a new member, so
@@ -1154,7 +1108,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
 
         // If it's a new sender, wait our turn, generate the event
         if (isNew) {
-          shunned = !addSurpriseMember(m, getStubForMember(m));
+          shunned = !addSurpriseMember(m);
         } // isNew
       }
 
@@ -1166,7 +1120,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     if (shunned) { // bug #41538 - shun notification must be outside synchronization to avoid hanging
       warnShun(m);
       logger.info("Membership: Ignoring message from shunned member <{}>:{}", m, msg);
-      throw new MemberShunnedException(getStubForMember(m));
+      throw new MemberShunnedException(m);
     }
     
     listener.messageReceived(msg);
@@ -1248,13 +1202,11 @@ public class GMSMembershipManager implements MembershipManager, Manager
    * grabbed a stable view if this is really a new member.
    * 
    * @param member
-   * @param stub
    */
   private void processSurpriseConnect(
-      InternalDistributedMember member, 
-      Stub stub) 
+      InternalDistributedMember member) 
   {
-    addSurpriseMember(member, stub);
+    addSurpriseMember(member);
   }
   
   /**
@@ -1276,7 +1228,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       processView(o.gmsView.getViewId(), o.gmsView);
     }
     else if (o.isSurpriseConnect()) { // connect
-      processSurpriseConnect(o.member, o.stub);
+      processSurpriseConnect(o.member);
     }
     
     else // sanity
@@ -1450,7 +1402,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     }
   }
 
-  public boolean memberExists(InternalDistributedMember m) {
+  public boolean memberExists(DistributedMember m) {
     latestViewLock.readLock().lock();
     NetView v = latestView;
     latestViewLock.readLock().unlock();
@@ -1525,12 +1477,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
       directChannel.emergencyClose();
     }
     
-    // could we guarantee not to allocate objects?  We're using Darrel's 
-    // factory, so it's possible that an unsafe implementation could be
-    // introduced here.
-//    stubToMemberMap.clear();
-//    memberToStubMap.clear();
-    
     if (DEBUG) {
       System.err.println("DEBUG: done closing GroupMembershipService");
     }
@@ -1767,7 +1713,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       allDestinations = true;
       latestViewLock.writeLock().lock();
       try {
-        Set keySet = memberToStubMap.keySet();
+        List<InternalDistributedMember> keySet = latestView.getMembers();
         keys = new InternalDistributedMember[keySet.size()];
         keys = (InternalDistributedMember[])keySet.toArray(keys);
       } finally {
@@ -2020,80 +1966,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     // not currently supported by this manager
   }
   
-  /**
-   * Get or create stub for given member
-   */
-  public Stub getStubForMember(InternalDistributedMember m)
-  {
-    if (shutdownInProgress) {
-      throw new DistributedSystemDisconnectedException(LocalizedStrings.GroupMembershipService_DISTRIBUTEDSYSTEM_IS_SHUTTING_DOWN.toLocalizedString(), services.getShutdownCause());
-    }
-
-    if (services.getConfig().getDistributionConfig().getDisableTcp()) {
-      return new Stub(m.getInetAddress(), m.getPort(), m.getVmViewId());
-    }
-    
-    // Return existing one if it is already in place
-    Stub result;
-    result = (Stub)memberToStubMap.get(m);
-    if (result != null)
-      return result;
-
-    latestViewLock.writeLock().lock();
-    try {
-      // Do all of this work in a critical region to prevent
-      // members from slipping in during shutdown
-      if (shutdownInProgress())
-        return null; // don't try to create a stub during shutdown
-      if (isShunned(m))
-        return null; // don't let zombies come back to life
-      
-      // OK, create one.  Update the table to reflect the creation.
-      result = directChannel.createConduitStub(m);
-      addChannel(m, result);
-    } finally {
-      latestViewLock.writeLock().unlock();
-    }
-   return result;
-  }
-
-  public InternalDistributedMember getMemberForStub(Stub s, boolean validated)
-  {
-    latestViewLock.writeLock().lock();
-    try {
-      if (shutdownInProgress) {
-        throw new DistributedSystemDisconnectedException(LocalizedStrings.GroupMembershipService_DISTRIBUTEDSYSTEM_IS_SHUTTING_DOWN.toLocalizedString(), services.getShutdownCause());
-      }
-      InternalDistributedMember result = (InternalDistributedMember)
-          stubToMemberMap.get(s);
-      if (result != null) {
-        if (validated && !this.latestView.contains(result)) {
-          // Do not return this member unless it is in the current view.
-          if (!surpriseMembers.containsKey(result)) {
-            // if not a surprise member, this stub is lingering and should be removed
-            stubToMemberMap.remove(s);
-            memberToStubMap.remove(result);
-          }
-          result = null;
-          // fall through to see if there is a newer member using the same direct port
-        }
-      }
-      if (result == null) {
-        // it may have not been added to the stub->idm map yet, so check the current view
-        for (InternalDistributedMember idm: latestView.getMembers()) {
-          if (GMSUtil.compareAddresses(idm.getInetAddress(), s.getInetAddress()) == 0
-              && idm.getDirectChannelPort() == s.getPort()) {
-            addChannel(idm, s);
-            return idm;
-          }
-        }
-      }
-      return result;
-    } finally {
-      latestViewLock.writeLock().unlock();
-    }
-  }
-
   public void setShutdown()
   {
     latestViewLock.writeLock().lock();
@@ -2109,24 +1981,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     return shutdownInProgress || (dm != null && dm.shutdownInProgress());
   }
   
-  /**
-   * Add a mapping from the given member to the given stub. Must
-   * be called with {@link #latestViewLock} held.
-   * 
-   * @param member
-   * @param theChannel
-   */
-  protected void addChannel(InternalDistributedMember member, Stub theChannel)
-  {
-    if (theChannel != null) {
-      // Don't overwrite existing stub information with a null
-      this.memberToStubMap.put(member, theChannel);
-
-      // Can't create reverse mapping if the stub is null
-      this.stubToMemberMap.put(theChannel, member);
-    }
-  }
-
 
   /**
    * Clean up and create consistent new view with member removed.
@@ -2137,12 +1991,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
   protected void destroyMember(final InternalDistributedMember member,
       boolean crashed, final String reason) {
     
-    // Clean up the maps
-    Stub theChannel = (Stub)memberToStubMap.remove(member);
-    if (theChannel != null) {
-      this.stubToMemberMap.remove(theChannel);
-    }
-    
     // Make sure it is removed from the view
     latestViewLock.writeLock().lock();
     try {
@@ -2365,12 +2213,11 @@ public class GMSMembershipManager implements MembershipManager, Manager
   /* non-thread-owned serial channels and high priority channels are not
    * included
    */
-  public Map getMessageState(DistributedMember member, boolean includeMulticast) {
+  public Map getChannelStates(DistributedMember member, boolean includeMulticast) {
     Map result = new HashMap();
-    Stub stub = (Stub)memberToStubMap.get(member);
     DirectChannel dc = directChannel;
-    if (stub != null && dc != null) {
-      dc.getChannelStates(stub, result);
+    if (dc != null) {
+      dc.getChannelStates(member, result);
     }
     services.getMessenger().getMessageState((InternalDistributedMember)member, result, includeMulticast);
     return result;
@@ -2381,15 +2228,8 @@ public class GMSMembershipManager implements MembershipManager, Manager
   {
     if (Thread.interrupted()) throw new InterruptedException();
     DirectChannel dc = directChannel;
-    Stub stub;
-    latestViewLock.writeLock().lock();
-    try {
-      stub = (Stub)memberToStubMap.get(otherMember);
-    } finally {
-      latestViewLock.writeLock().unlock();
-    }
-    if (dc != null && stub != null) {
-      dc.waitForChannelState(stub, state);
+    if (dc != null) {
+      dc.waitForChannelState(otherMember, channelState);
     }
     services.getMessenger().waitForMessageState((InternalDistributedMember)otherMember, state);
   }
@@ -2405,7 +2245,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     boolean result = false;
     DirectChannel dc = directChannel;
     InternalDistributedMember idm = (InternalDistributedMember)mbr;
-    Stub stub = new Stub(idm.getInetAddress(), idm.getPort(), idm.getVmViewId());
     int memberTimeout = this.services.getConfig().getDistributionConfig().getMemberTimeout();
     long pauseTime = (memberTimeout < 1000) ? 100 : memberTimeout / 10;
     boolean wait;
@@ -2413,7 +2252,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     do {
       wait = false;
       if (dc != null) {
-        if (dc.hasReceiversFor(stub)) {
+        if (dc.hasReceiversFor(idm)) {
           wait = true;
         }
         if (wait && logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/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 7bb97b9..780fe18 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
@@ -1109,7 +1109,7 @@ class ParentLocalizedStrings {
   public static final StringId TCPConduit_ENDING_RECONNECT_ATTEMPT_BECAUSE_0_HAS_DISAPPEARED = new StringId(2086, "Ending reconnect attempt because {0} has disappeared.");
   public static final StringId TCPConduit_ENDING_RECONNECT_ATTEMPT_TO_0_BECAUSE_SHUTDOWN_HAS_STARTED = new StringId(2087, "Ending reconnect attempt to {0} because shutdown has started.");
   public static final StringId TCPConduit_ERROR_SENDING_MESSAGE_TO_0_WILL_REATTEMPT_1 = new StringId(2088, "Error sending message to {0} (will reattempt): {1}");
-  public static final StringId TCPConduit_EXCEPTION_CREATING_SERVERSOCKET = new StringId(2089, "While creating ServerSocket and Stub on port {0} with address {1}");
+  public static final StringId TCPConduit_EXCEPTION_CREATING_SERVERSOCKET = new StringId(2089, "While creating ServerSocket on port {0} with address {1}");
   public static final StringId TCPConduit_EXCEPTION_PARSING_P2PIDLECONNECTIONTIMEOUT = new StringId(2090, "exception parsing p2p.idleConnectionTimeout");
   public static final StringId TCPConduit_EXCEPTION_PARSING_P2PTCPBUFFERSIZE = new StringId(2091, "exception parsing p2p.tcpBufferSize");
   public static final StringId TCPConduit_FAILED_TO_ACCEPT_CONNECTION_FROM_0_BECAUSE_1 = new StringId(2092, "Failed to accept connection from {0} because {1}");
@@ -1444,7 +1444,7 @@ class ParentLocalizedStrings {
   public static final StringId Connection_DETECTED_WRONG_VERSION_OF_GEMFIRE_PRODUCT_DURING_HANDSHAKE_EXPECTED_0_BUT_FOUND_1 = new StringId(2432, "Detected wrong version of GemFire product during handshake. Expected  {0}  but found  {1}");
   public static final StringId Connection_FORCED_DISCONNECT_SENT_TO_0 = new StringId(2433, "Forced disconnect sent to  {0}");
   public static final StringId Connection_HANDSHAKE_FAILED = new StringId(2434, "Handshake failed");
-  public static final StringId Connection_MEMBER_FOR_STUB_0_LEFT_THE_GROUP = new StringId(2435, "Member for stub  {0}  left the group");
+  public static final StringId Connection_MEMBER_LEFT_THE_GROUP = new StringId(2435, "Member {0}  left the group");
   public static final StringId Connection_NOT_CONNECTED_TO_0 = new StringId(2436, "Not connected to  {0}");
   public static final StringId Connection_NULL_CONNECTIONTABLE = new StringId(2437, "Null ConnectionTable");
   public static final StringId Connection_SOCKET_HAS_BEEN_CLOSED = new StringId(2438, "socket has been closed");
@@ -1542,7 +1542,7 @@ class ParentLocalizedStrings {
   public static final StringId DefaultQuery_WHEN_QUERYING_A_PARTITIONED_REGION_THE_PROJECTIONS_MUST_NOT_REFERENCE_ANY_REGIONS = new StringId(2530, "When querying a Partitioned Region, the projections must not reference any regions");
   public static final StringId DestroyMessage_FAILED_SENDING_0 = new StringId(2531, "Failed sending < {0} >");
   public static final StringId DirectChannel_COMMUNICATIONS_DISCONNECTED = new StringId(2532, "communications disconnected");
-  public static final StringId DirectChannel_NO_STUB_0 = new StringId(2533, "No stub {0}");
+  public static final StringId DirectChannel_SHUNNING_0 = new StringId(2533, "Member is being shunned: {0}");
   public static final StringId DirectChannel_UNKNOWN_ERROR_SERIALIZING_MESSAGE = new StringId(2534, "Unknown error serializing message");
   public static final StringId DiskEntry_AN_IOEXCEPTION_WAS_THROWN_WHILE_SERIALIZING = new StringId(2535, "An IOException was thrown while serializing.");
   public static final StringId DiskEntry_DISK_REGION_IS_NULL = new StringId(2536, "Disk region is null");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
index f918812..74660da 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
@@ -50,6 +50,7 @@ import org.apache.logging.log4j.Logger;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.ConflationKey;
 import com.gemstone.gemfire.distributed.internal.DM;
@@ -72,7 +73,6 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.ByteArrayDataInput;
 import com.gemstone.gemfire.internal.DSFIDFactory;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
-import com.gemstone.gemfire.internal.SocketCloser;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.SystemTimer;
@@ -222,11 +222,6 @@ public class Connection implements Runnable {
   /** the ID string of the conduit (for logging) */
   String conduitIdStr;
 
-  /** remoteId identifies the remote conduit's listener.  It does NOT
-     identify the "port" that this connection's socket is attached
-     to, which is a different thing altogether */
-  Stub remoteId;
-
   /** Identifies the java group member on the other side of the connection. */
   InternalDistributedMember remoteAddr;
 
@@ -801,7 +796,7 @@ public class Connection implements Runnable {
           }
           if (success) {
             if (this.isReceiver) {
-              needToClose = !owner.getConduit().getMembershipManager().addSurpriseMember(this.remoteAddr, this.remoteId);
+              needToClose = !owner.getConduit().getMembershipManager().addSurpriseMember(this.remoteAddr);
               if (needToClose) {
                 reason = "this member is shunned";
               }
@@ -845,7 +840,7 @@ public class Connection implements Runnable {
    * @param beingSick
    */
   private void asyncClose(boolean beingSick) {
-    // note: remoteId may be null if this is a receiver that hasn't finished its handshake
+    // note: remoteAddr may be null if this is a receiver that hasn't finished its handshake
     
     // we do the close in a background thread because the operation may hang if 
     // there is a problem with the network.  See bug #46659
@@ -1018,8 +1013,7 @@ public class Connection implements Runnable {
   protected static Connection createSender(final MembershipManager mgr,
                                            final ConnectionTable t,
                                            final boolean preserveOrder,
-                                           final Stub key,
-                                           final InternalDistributedMember remoteAddr,
+                                           final DistributedMember remoteAddr,
                                            final boolean sharedResource,
                                            final long startTime,
                                            final long ackTimeout,
@@ -1074,9 +1068,8 @@ public class Connection implements Runnable {
         }
         if (firstTime) {
           firstTime = false;
-          InternalDistributedMember m = mgr.getMemberForStub(key, true);
-          if (m == null) {
-            throw new IOException("Member for stub " + key + " left the group");
+          if (!mgr.memberExists(remoteAddr) || mgr.isShunned(remoteAddr) || mgr.shutdownInProgress()) {
+            throw new IOException("Member " + remoteAddr + " left the system");
           }
         }
         else {
@@ -1084,7 +1077,7 @@ public class Connection implements Runnable {
           // alert listener should not prevent cache operations from continuing
           if (AlertAppender.isThreadAlerting()) {
             // do not change the text of this exception - it is looked for in exception handlers
-            throw new IOException("Cannot form connection to alert listener " + key);
+            throw new IOException("Cannot form connection to alert listener " + remoteAddr);
           }
             
           // Wait briefly...
@@ -1097,20 +1090,19 @@ public class Connection implements Runnable {
             t.getConduit().getCancelCriterion().checkCancelInProgress(ie);
           }
           t.getConduit().getCancelCriterion().checkCancelInProgress(null);
-          InternalDistributedMember m = mgr.getMemberForStub(key, true);
-          if (m == null) {
-            throw new IOException(LocalizedStrings.Connection_MEMBER_FOR_STUB_0_LEFT_THE_GROUP.toLocalizedString(key));
+          if (giveUpOnMember(mgr, remoteAddr)) {
+            throw new IOException(LocalizedStrings.Connection_MEMBER_LEFT_THE_GROUP.toLocalizedString(remoteAddr));
           }
           if (!warningPrinted) {
             warningPrinted = true;
-            logger.warn(LocalizedMessage.create(LocalizedStrings.Connection_CONNECTION_ATTEMPTING_RECONNECT_TO_PEER__0, m));
+            logger.warn(LocalizedMessage.create(LocalizedStrings.Connection_CONNECTION_ATTEMPTING_RECONNECT_TO_PEER__0, remoteAddr));
           }          
           t.getConduit().stats.incReconnectAttempts();
         }
         //create connection
         try {
           conn = null;
-          conn = new Connection(mgr, t, preserveOrder, key, remoteAddr, sharedResource);
+          conn = new Connection(mgr, t, preserveOrder, remoteAddr, sharedResource);
         }
         catch (javax.net.ssl.SSLHandshakeException se) {
           // no need to retry if certificates were rejected
@@ -1118,8 +1110,7 @@ public class Connection implements Runnable {
         }
         catch (IOException ioe) {
           // Only give up if the member leaves the view.
-          InternalDistributedMember m = mgr.getMemberForStub(key, true);
-          if (m == null) {
+          if (giveUpOnMember(mgr, remoteAddr)) {
             throw ioe;
           }
           t.getConduit().getCancelCriterion().checkCancelInProgress(null);
@@ -1130,7 +1121,7 @@ public class Connection implements Runnable {
             connectionErrorLogged = true; // otherwise change to use 100ms intervals causes a lot of these
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.Connection_CONNECTION_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1,
-                new Object[] {sharedResource, preserveOrder, m, ioe}));
+                new Object[] {sharedResource, preserveOrder, remoteAddr, ioe}));
           }
         } // IOException
         finally {
@@ -1146,9 +1137,8 @@ public class Connection implements Runnable {
               // something went wrong while reading the handshake
               // and the socket was closed or this guy sent us a
               // ShutdownMessage
-              InternalDistributedMember m = mgr.getMemberForStub(key, true);
-              if (m == null) {
-                throw new IOException(LocalizedStrings.Connection_MEMBER_FOR_STUB_0_LEFT_THE_GROUP.toLocalizedString(key));
+              if (giveUpOnMember(mgr, remoteAddr)) {
+                throw new IOException(LocalizedStrings.Connection_MEMBER_LEFT_THE_GROUP.toLocalizedString(remoteAddr));
               }
               t.getConduit().getCancelCriterion().checkCancelInProgress(null);
               // no success but no need to log; just retry
@@ -1161,8 +1151,7 @@ public class Connection implements Runnable {
             throw e;
           }
           catch (ConnectionException e) {
-            InternalDistributedMember m = mgr.getMemberForStub(key, true);
-            if (m == null) {
+            if (giveUpOnMember(mgr, remoteAddr)) {
               IOException ioe = new IOException(LocalizedStrings.Connection_HANDSHAKE_FAILED.toLocalizedString());
               ioe.initCause(e);
               throw ioe;
@@ -1170,17 +1159,16 @@ public class Connection implements Runnable {
             t.getConduit().getCancelCriterion().checkCancelInProgress(null);
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.Connection_CONNECTION_HANDSHAKE_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1,
-                new Object[] {sharedResource, preserveOrder, m,e}));
+                new Object[] {sharedResource, preserveOrder, remoteAddr ,e}));
           }
           catch (IOException e) {
-            InternalDistributedMember m = mgr.getMemberForStub(key, true);
-            if (m == null) {
+            if (giveUpOnMember(mgr, remoteAddr)) {
               throw e;
             }
             t.getConduit().getCancelCriterion().checkCancelInProgress(null);
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.Connection_CONNECTION_HANDSHAKE_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1,
-                new Object[] {sharedResource, preserveOrder, m,e}));
+                new Object[] {sharedResource, preserveOrder, remoteAddr ,e}));
             if (!sharedResource && "Too many open files".equals(e.getMessage())) {
               t.fileDescriptorsExhausted();
             }
@@ -1220,7 +1208,7 @@ public class Connection implements Runnable {
     if (conn == null) {
       throw new ConnectionException(
         LocalizedStrings.Connection_CONNECTION_FAILED_CONSTRUCTION_FOR_PEER_0
-          .toLocalizedString(mgr.getMemberForStub(key, true)));
+          .toLocalizedString(remoteAddr));
     }
     if (preserveOrder && BATCH_SENDS) {
       conn.createBatchSendBuffer();
@@ -1228,12 +1216,15 @@ public class Connection implements Runnable {
     conn.finishedConnecting = true;
     return conn;
   }
+  
+  private static boolean giveUpOnMember(MembershipManager mgr, DistributedMember remoteAddr) {
+    return !mgr.memberExists(remoteAddr) || mgr.isShunned(remoteAddr) || mgr.shutdownInProgress();
+  }
 
-  private void setRemoteAddr(InternalDistributedMember m, Stub stub) {
+  private void setRemoteAddr(DistributedMember m) {
     this.remoteAddr = this.owner.getDM().getCanonicalId(m);
-    this.remoteId = stub;
     MembershipManager mgr = this.owner.owner.getMembershipManager();
-    mgr.addSurpriseMember(m, stub);
+    mgr.addSurpriseMember(m);
   }
   
   /** creates a new connection to a remote server.
@@ -1243,11 +1234,11 @@ public class Connection implements Runnable {
   private Connection(MembershipManager mgr,
                      ConnectionTable t,
                      boolean preserveOrder,
-                     Stub key,
-                     InternalDistributedMember remoteAddr,
+                     DistributedMember remoteID,
                      boolean sharedResource)
     throws IOException, DistributedSystemDisconnectedException
   {    
+    InternalDistributedMember remoteAddr = (InternalDistributedMember)remoteID;
     if (t == null) {
       throw new IllegalArgumentException(LocalizedStrings.Connection_CONNECTIONTABLE_IS_NULL.toLocalizedString());
     }
@@ -1255,7 +1246,7 @@ public class Connection implements Runnable {
     this.owner = t;
     this.sharedResource = sharedResource;
     this.preserveOrder = preserveOrder;
-    setRemoteAddr(remoteAddr, key);
+    setRemoteAddr(remoteAddr);
     this.conduitIdStr = this.owner.getConduit().getId().toString();
     this.handshakeRead = false;
     this.handshakeCancelled = false;
@@ -1265,7 +1256,7 @@ public class Connection implements Runnable {
 
     // connect to listening socket
 
-    InetSocketAddress addr = new InetSocketAddress(remoteId.getInetAddress(), remoteId.getPort());
+    InetSocketAddress addr = new InetSocketAddress(remoteAddr.getInetAddress(), remoteAddr.getDirectChannelPort());
     if (useNIO()) {
       SocketChannel channel = SocketChannel.open();
       this.owner.addConnectingSocket(channel.socket(), addr.getAddress());
@@ -1325,15 +1316,15 @@ public class Connection implements Runnable {
     else {
       if (TCPConduit.useSSL) {
         // socket = javax.net.ssl.SSLSocketFactory.getDefault()
-        //  .createSocket(remoteId.getInetAddress(), remoteId.getPort());
+        //  .createSocket(remoteAddr.getInetAddress(), remoteAddr.getPort());
         int socketBufferSize = sharedResource ? SMALL_BUFFER_SIZE : this.owner.getConduit().tcpBufferSize;
-        this.socket = SocketCreator.getDefaultInstance().connectForServer( remoteId.getInetAddress(), remoteId.getPort(), socketBufferSize );
+        this.socket = SocketCreator.getDefaultInstance().connectForServer( remoteAddr.getInetAddress(), remoteAddr.getDirectChannelPort(), socketBufferSize );
         // Set the receive buffer size local fields. It has already been set in the socket.
         setSocketBufferSize(this.socket, false, socketBufferSize, true);
         setSendBufferSize(this.socket);
       }
       else {
-        //socket = new Socket(remoteId.getInetAddress(), remoteId.getPort());
+        //socket = new Socket(remoteAddr.getInetAddress(), remoteAddr.getPort());
         Socket s = new Socket();
         this.socket = s;
         s.setTcpNoDelay(true);
@@ -1639,8 +1630,8 @@ public class Connection implements Runnable {
     // we can't wait for the reader thread when running in an IBM JRE.  See
     // bug 41889
     if (this.owner.owner.config.getEnableNetworkPartitionDetection() ||
-        this.owner.owner.getLocalId().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE ||
-        this.owner.owner.getLocalId().getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
+        this.owner.owner.getLocalAddr().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE ||
+        this.owner.owner.getLocalAddr().getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
       isIBM = "IBM Corporation".equals(System.getProperty("java.vm.vendor"));
     }
     {
@@ -1689,16 +1680,16 @@ public class Connection implements Runnable {
               // Only remove endpoint if sender.
               if (this.finishedConnecting) {
                 // only remove endpoint if our constructor finished
-                this.owner.removeEndpoint(this.remoteId, reason);
+                this.owner.removeEndpoint(this.remoteAddr, reason);
               }
             }
           }
           else {
-            this.owner.removeSharedConnection(reason, this.remoteId, this.preserveOrder, this);
+            this.owner.removeSharedConnection(reason, this.remoteAddr, this.preserveOrder, this);
           }
         }
         else if (!this.isReceiver) {
-          this.owner.removeThreadConnection(this.remoteId, this);
+          this.owner.removeThreadConnection(this.remoteAddr, this);
         }
       }
       else {
@@ -1706,10 +1697,10 @@ public class Connection implements Runnable {
         // has never added this Connection to its maps since
         // the calls in this block use our identity to do the removes.
         if (this.sharedResource) {
-          this.owner.removeSharedConnection(reason, this.remoteId, this.preserveOrder, this);
+          this.owner.removeSharedConnection(reason, this.remoteAddr, this.preserveOrder, this);
         }
         else if (!this.isReceiver) {
-          this.owner.removeThreadConnection(this.remoteId, this);
+          this.owner.removeThreadConnection(this.remoteAddr, this);
         }
       }
     }
@@ -1753,7 +1744,7 @@ public class Connection implements Runnable {
     } finally {
       // bug36060: do the socket close within a finally block
       if (logger.isDebugEnabled()) {
-        logger.debug("Stopping {} for {}", p2pReaderName(), remoteId);
+        logger.debug("Stopping {} for {}", p2pReaderName(), remoteAddr);
       }
       initiateSuspicionIfSharedUnordered();
       if (this.isReceiver) {
@@ -2338,8 +2329,7 @@ public class Connection implements Runnable {
                     .toLocalizedString(new Object[]{new Byte(HANDSHAKE_VERSION), new Byte(handShakeByte)}));
               }
               InternalDistributedMember remote = DSFIDFactory.readInternalDistributedMember(dis);
-              Stub stub = new Stub(remote.getInetAddress()/*fix for bug 33615*/, remote.getDirectChannelPort(), remote.getVmViewId());
-              setRemoteAddr(remote, stub);
+              setRemoteAddr(remote);
               Thread.currentThread().setName(LocalizedStrings.Connection_P2P_MESSAGE_READER_FOR_0.toLocalizedString(this.remoteAddr, this.socket.getPort()));
               this.sharedResource = dis.readBoolean();
               this.preserveOrder = dis.readBoolean();
@@ -2377,7 +2367,7 @@ public class Connection implements Runnable {
               }
               
               if (logger.isDebugEnabled()) {
-                logger.debug("{} remoteId is {} {}", p2pReaderName(), this.remoteId,
+                logger.debug("{} remoteAddr is {} {}", p2pReaderName(), this.remoteAddr,
                     (this.remoteVersion != null ? " (" + this.remoteVersion + ')' : ""));
               }
 
@@ -2555,7 +2545,7 @@ public class Connection implements Runnable {
     throws IOException, ConnectionException
   {
     if (!connected) {
-      throw new ConnectionException(LocalizedStrings.Connection_NOT_CONNECTED_TO_0.toLocalizedString(this.remoteId));
+      throw new ConnectionException(LocalizedStrings.Connection_NOT_CONNECTED_TO_0.toLocalizedString(this.remoteAddr));
     }
     if (this.batchFlusher != null) {
       batchSend(buffer);
@@ -2778,7 +2768,7 @@ public class Connection implements Runnable {
         if (this.disconnectRequested) {
           buffer.position(origBufferPos);
           // we have given up so just drop this message.
-          throw new ConnectionException(LocalizedStrings.Connection_FORCED_DISCONNECT_SENT_TO_0.toLocalizedString(this.remoteId));
+          throw new ConnectionException(LocalizedStrings.Connection_FORCED_DISCONNECT_SENT_TO_0.toLocalizedString(this.remoteAddr));
         }
         if (!force && !this.asyncQueuingInProgress) {
           // reset buffer since we will be sending it. This fixes bug 34832
@@ -2980,7 +2970,7 @@ public class Connection implements Runnable {
     }
     DM dm = this.owner.getDM();
     if (dm == null) {
-      this.owner.removeEndpoint(this.remoteId, LocalizedStrings.Connection_NO_DISTRIBUTION_MANAGER.toLocalizedString());
+      this.owner.removeEndpoint(this.remoteAddr, LocalizedStrings.Connection_NO_DISTRIBUTION_MANAGER.toLocalizedString());
       return;
     }
     dm.getMembershipManager().requestMemberRemoval(this.remoteAddr, 
@@ -3001,7 +2991,7 @@ public class Connection implements Runnable {
         return;
       }
     }
-    this.owner.removeEndpoint(this.remoteId, 
+    this.owner.removeEndpoint(this.remoteAddr, 
                               LocalizedStrings.Connection_FORCE_DISCONNECT_TIMED_OUT.toLocalizedString());
     if (dm.getOtherDistributionManagerIds().contains(this.remoteAddr)) {
       if (logger.isDebugEnabled()) {
@@ -3110,7 +3100,7 @@ public class Connection implements Runnable {
       stats.incAsyncThreads(-1);
       stats.incAsyncQueues(-1);
       if (logger.isDebugEnabled()) {
-        logger.debug("runNioPusher terminated id={} from {}/{}", conduitIdStr, remoteId, remoteAddr);
+        logger.debug("runNioPusher terminated id={} from {}/{}", conduitIdStr, remoteAddr, remoteAddr);
       }
     }
     } finally {
@@ -3837,8 +3827,7 @@ public class Connection implements Runnable {
                   throw new IllegalStateException(LocalizedStrings.Connection_DETECTED_WRONG_VERSION_OF_GEMFIRE_PRODUCT_DURING_HANDSHAKE_EXPECTED_0_BUT_FOUND_1.toLocalizedString(new Object[] {new Byte(HANDSHAKE_VERSION), new Byte(handShakeByte)}));
                 }
                 InternalDistributedMember remote = DSFIDFactory.readInternalDistributedMember(dis);
-                Stub stub = new Stub(remote.getInetAddress()/*fix for bug 33615*/, remote.getDirectChannelPort(), remote.getVmViewId());
-                setRemoteAddr(remote, stub);
+                setRemoteAddr(remote);
                 this.sharedResource = dis.readBoolean();
                 this.preserveOrder = dis.readBoolean();
                 this.uniqueId = dis.readLong();
@@ -3897,7 +3886,7 @@ public class Connection implements Runnable {
                 return;
               }
               if (logger.isDebugEnabled()) {
-                logger.debug("P2P handshake remoteId is {}{}", this.remoteId,
+                logger.debug("P2P handshake remoteAddr is {}{}", this.remoteAddr,
                     (this.remoteVersion != null ? " (" + this.remoteVersion + ')' : ""));
               }
               try {
@@ -4031,12 +4020,6 @@ public class Connection implements Runnable {
     this.accessed = true;
   }
 
-  /** returns the ConnectionKey stub representing the other side of
-      this connection (host:port) */
-  public final Stub getRemoteId() {
-    return remoteId;
-  }
-
   /** return the DM id of the guy on the other side of this connection.
    */
   public final InternalDistributedMember getRemoteAddress() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
index bac356c..3816efe 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
@@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
@@ -60,7 +61,7 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 
 /** <p>ConnectionTable holds all of the Connection objects in a conduit.
     Connections represent a pipe between two endpoints represented
-    by generic Stubs.</p>
+    by generic DistributedMembers.</p>
 
     @author Bruce Schuchardt
     @author Darrel Schneider
@@ -345,7 +346,7 @@ public class ConnectionTable  {
   /**
    * Process a newly created PendingConnection
    * 
-   * @param id Stub on which the connection is created
+   * @param id DistributedMember on which the connection is created
    * @param sharedResource whether the connection is used by multiple threads
    * @param preserveOrder whether to preserve order
    * @param m map to add the connection to
@@ -357,7 +358,7 @@ public class ConnectionTable  {
    * @throws IOException if unable to connect
    * @throws DistributedSystemDisconnectedException
    */
-  private Connection handleNewPendingConnection(Stub id, boolean sharedResource,
+  private Connection handleNewPendingConnection(DistributedMember id, boolean sharedResource,
       boolean preserveOrder,
       Map m, PendingConnection pc, long startTime, long ackThreshold, long ackSAThreshold)
       throws IOException, DistributedSystemDisconnectedException
@@ -366,7 +367,7 @@ public class ConnectionTable  {
     Connection con = null;
     try {
       con = Connection.createSender(owner.getMembershipManager(), this, preserveOrder,
-                                    id, this.owner.getMemberForStub(id, false),
+                                    id,
                                     sharedResource,
                                     startTime, ackThreshold, ackSAThreshold);
       this.owner.stats.incSenders(sharedResource, preserveOrder);
@@ -442,7 +443,7 @@ public class ConnectionTable  {
    * unordered or conserve-sockets
    * note that unordered connections are currently always shared
    * 
-   * @param id the Stub on which we are creating a connection
+   * @param id the DistributedMember on which we are creating a connection
    * @param threadOwnsResources whether unordered conn is owned by the current thread
    * @param preserveOrder whether to preserve order
    * @param startTime the ms clock start time for the operation
@@ -452,7 +453,7 @@ public class ConnectionTable  {
    * @throws IOException if unable to create the connection
    * @throws DistributedSystemDisconnectedException
    */
-  private Connection getUnorderedOrConserveSockets(Stub id, 
+  private Connection getUnorderedOrConserveSockets(DistributedMember id, 
       boolean threadOwnsResources, boolean preserveOrder,
       long startTime, long ackTimeout, long ackSATimeout)
     throws IOException, DistributedSystemDisconnectedException
@@ -527,7 +528,7 @@ public class ConnectionTable  {
    * @throws IOException if the connection could not be created
    * @throws DistributedSystemDisconnectedException
    */
-  Connection getOrderedAndOwned(Stub id, long startTime, long ackTimeout, long ackSATimeout) 
+  Connection getOrderedAndOwned(DistributedMember id, long startTime, long ackTimeout, long ackSATimeout) 
       throws IOException, DistributedSystemDisconnectedException  {
     Connection result = null;
     
@@ -566,7 +567,7 @@ public class ConnectionTable  {
     // OK, we have to create a new connection.
     result = Connection.createSender(owner.getMembershipManager(), 
         this, true /* preserveOrder */, id,
-        this.owner.getMemberForStub(id, false), false /* shared */,
+        false /* shared */,
         startTime, ackTimeout, ackSATimeout);
     if (logger.isDebugEnabled()) {
       logger.debug("ConnectionTable: created an ordered connection: {}", result);
@@ -583,7 +584,7 @@ public class ConnectionTable  {
     
     ArrayList al = (ArrayList)this.threadConnectionMap.get(id);
     if (al == null) {
-      // First connection for this Stub.  Make sure list for this
+      // First connection for this DistributedMember.  Make sure list for this
       // stub is created if it isn't already there.
       al = new ArrayList();
       
@@ -651,7 +652,7 @@ public class ConnectionTable  {
   
   /**
    * Get a new connection
-   * @param id the Stub on which to create the connection
+   * @param id the DistributedMember on which to create the connection
    * @param preserveOrder whether order should be preserved
    * @param startTime the ms clock start time
    * @param ackTimeout the ms ack-wait-threshold, or zero
@@ -660,7 +661,7 @@ public class ConnectionTable  {
    * @throws java.io.IOException if the connection could not be created
    * @throws DistributedSystemDisconnectedException
    */
-  protected Connection get(Stub id, boolean preserveOrder,
+  protected Connection get(DistributedMember id, boolean preserveOrder,
       long startTime, long ackTimeout, long ackSATimeout) 
       throws java.io.IOException, DistributedSystemDisconnectedException
   {
@@ -838,34 +839,38 @@ public class ConnectionTable  {
   /**
    * Return true if our owner already knows that this endpoint is departing 
    */
-  protected boolean isEndpointShuttingDown(Stub stub) {
-    return this.owner.getMemberForStub(stub, true) == null;
+  protected boolean isEndpointShuttingDown(DistributedMember id) {
+    return giveUpOnMember(owner.getDM().getMembershipManager(), id);
   }
   
+  protected boolean giveUpOnMember(MembershipManager mgr, DistributedMember remoteAddr) {
+    return !mgr.memberExists(remoteAddr) || mgr.isShunned(remoteAddr) || mgr.shutdownInProgress();
+  }
+
   /** remove an endpoint and notify the membership manager of the departure */
-  protected void removeEndpoint(Stub stub, String reason) {
+  protected void removeEndpoint(DistributedMember stub, String reason) {
     removeEndpoint(stub, reason, true);
   }
 
-  protected void removeEndpoint(Stub stub, String reason, boolean notifyDisconnect) {
+  protected void removeEndpoint(DistributedMember memberID, String reason, boolean notifyDisconnect) {
     if (this.closed) {
       return;
     }
     boolean needsRemoval = false;
     synchronized (this.orderedConnectionMap) {
-      if (this.orderedConnectionMap.get(stub) != null)
+      if (this.orderedConnectionMap.get(memberID) != null)
         needsRemoval = true;
     }
     if (!needsRemoval) {
       synchronized (this.unorderedConnectionMap) {
-        if (this.unorderedConnectionMap.get(stub) != null)
+        if (this.unorderedConnectionMap.get(memberID) != null)
           needsRemoval = true;
       }
     }
     if (!needsRemoval) {
       ConcurrentMap cm = this.threadConnectionMap;
       if (cm != null) {
-        ArrayList al = (ArrayList)cm.get(stub);
+        ArrayList al = (ArrayList)cm.get(memberID);
         needsRemoval = al != null && al.size() > 0;
       }
     }
@@ -873,14 +878,14 @@ public class ConnectionTable  {
     if (needsRemoval) {
       InternalDistributedMember remoteAddress = null;
       synchronized (this.orderedConnectionMap) {
-        Object c = this.orderedConnectionMap.remove(stub);
+        Object c = this.orderedConnectionMap.remove(memberID);
         if (c instanceof Connection) {
           remoteAddress = ((Connection) c).getRemoteAddress();
         }
         closeCon(reason, c);
       }
       synchronized (this.unorderedConnectionMap) {
-        Object c = this.unorderedConnectionMap.remove(stub);
+        Object c = this.unorderedConnectionMap.remove(memberID);
         if (remoteAddress == null && (c instanceof Connection)) {
           remoteAddress = ((Connection) c).getRemoteAddress();
         }
@@ -890,7 +895,7 @@ public class ConnectionTable  {
       {
         ConcurrentMap cm = this.threadConnectionMap;
         if (cm != null) {
-          ArrayList al = (ArrayList)cm.remove(stub);
+          ArrayList al = (ArrayList)cm.remove(memberID);
           if (al != null) {
             synchronized (al) {
               for (Iterator it=al.iterator(); it.hasNext();) {
@@ -912,7 +917,7 @@ public class ConnectionTable  {
         for (Iterator it=connectingSockets.entrySet().iterator(); it.hasNext(); ) {
           Map.Entry entry = (Map.Entry)it.next();
           ConnectingSocketInfo info = (ConnectingSocketInfo)entry.getValue();
-          if (info.peerAddress.equals(stub.getInetAddress())) {
+          if (info.peerAddress.equals(((InternalDistributedMember)memberID).getInetAddress())) {
             toRemove.add(entry.getKey());
             it.remove();
           }
@@ -925,7 +930,7 @@ public class ConnectionTable  {
         }
         catch (IOException e) {
           if (logger.isDebugEnabled()) {
-            logger.debug("caught exception while trying to close connecting socket for {}", stub, e);
+            logger.debug("caught exception while trying to close connecting socket for {}", memberID, e);
           }
         }
       }
@@ -937,7 +942,7 @@ public class ConnectionTable  {
       synchronized (this.receivers) {
         for (Iterator it=receivers.iterator(); it.hasNext();) {
           Connection con = (Connection)it.next();
-          if (stub.equals(con.getRemoteId())) {
+          if (memberID.equals(con.getRemoteAddress())) {
             it.remove();
             toRemove.add(con);
           }
@@ -947,10 +952,13 @@ public class ConnectionTable  {
         Connection con = (Connection)it.next();
         closeCon(reason, con);
       }
-      // call memberDeparted after doing the closeCon calls
-      // so it can recursively call removeEndpoint
       if (notifyDisconnect) {
-        owner.getMemberForStub(stub, false);
+        // Before the removal of TCPConduit Stub addresses this used
+        // to call MembershipManager.getMemberForStub, which checked
+        // for a shutdown in progress and threw this exception:
+        if (owner.getDM().shutdownInProgress()) {
+          throw new DistributedSystemDisconnectedException("Shutdown in progress", owner.getDM().getMembershipManager().getShutdownCause());
+        }
       }
       
       if (remoteAddress != null) {
@@ -964,11 +972,11 @@ public class ConnectionTable  {
   }
   
   /** check to see if there are still any receiver threads for the given end-point */
-  protected boolean hasReceiversFor(Stub endPoint) {
+  protected boolean hasReceiversFor(DistributedMember endPoint) {
     synchronized (this.receivers) {
       for (Iterator it=receivers.iterator(); it.hasNext();) {
         Connection con = (Connection)it.next();
-        if (endPoint.equals(con.getRemoteId())) {
+        if (endPoint.equals(con.getRemoteAddress())) {
           return true;
         }
       }
@@ -976,7 +984,7 @@ public class ConnectionTable  {
     return false;
   }
   
-  private static void removeFromThreadConMap(ConcurrentMap cm, Stub stub, Connection c) {
+  private static void removeFromThreadConMap(ConcurrentMap cm, DistributedMember stub, Connection c) {
     if (cm != null) {
       ArrayList al = (ArrayList)cm.get(stub);
       if (al != null) {
@@ -986,7 +994,7 @@ public class ConnectionTable  {
       }
     }
   }
-  protected void removeThreadConnection(Stub stub, Connection c) {
+  protected void removeThreadConnection(DistributedMember stub, Connection c) {
     /*if (this.closed) {
       return;
     }*/
@@ -1001,7 +1009,7 @@ public class ConnectionTable  {
       } // synchronized
     } // m != null
   }
-  void removeSharedConnection(String reason, Stub stub, boolean ordered, Connection c) {
+  void removeSharedConnection(String reason, DistributedMember stub, boolean ordered, Connection c) {
     if (this.closed) {
       return;
     }
@@ -1054,7 +1062,7 @@ public class ConnectionTable  {
        Iterator it = m.entrySet().iterator();
        while (it.hasNext()) {
          Map.Entry me = (Map.Entry)it.next();
-         Stub stub = (Stub)me.getKey();
+         DistributedMember stub = (DistributedMember)me.getKey();
          Connection c = (Connection)me.getValue();
          removeFromThreadConMap(this.threadConnectionMap, stub, c);
          it.remove();
@@ -1079,7 +1087,7 @@ public class ConnectionTable  {
    * from being formed or new messages from being sent
    * @since 5.1
    */
-  protected void getThreadOwnedOrderedConnectionState(Stub member,
+  protected void getThreadOwnedOrderedConnectionState(DistributedMember member,
       Map result) {
 
     ConcurrentMap cm = this.threadConnectionMap;
@@ -1105,7 +1113,7 @@ public class ConnectionTable  {
    * wait for the given incoming connections to receive at least the associated
    * number of messages
    */
-  protected void waitForThreadOwnedOrderedConnectionState(Stub member,
+  protected void waitForThreadOwnedOrderedConnectionState(DistributedMember member,
       Map connectionStates) throws InterruptedException {
     if (Thread.interrupted()) throw new InterruptedException(); // wisest to do this before the synchronize below
     List r = null;
@@ -1115,14 +1123,14 @@ public class ConnectionTable  {
     for (Iterator it=r.iterator(); it.hasNext();) {
       Connection con = (Connection)it.next();
       if (!con.stopped && !con.isClosing() && !con.getOriginatedHere() && con.getPreserveOrder()
-          && member.equals(con.getRemoteId())) {
+          && member.equals(con.getRemoteAddress())) {
         Long state = (Long)connectionStates.remove(Long.valueOf(con.getUniqueId()));
         if (state != null) {
           long count = state.longValue();
           while (!con.stopped && !con.isClosing() && con.getMessagesReceived() < count) {
             if (logger.isDebugEnabled()) {
               logger.debug("Waiting for connection {}/{} currently={} need={}", 
-                  con.getRemoteId(), con.getUniqueId(), con.getMessagesReceived(), count);
+                  con.getRemoteAddress(), con.getUniqueId(), con.getMessagesReceived(), count);
             }
             Thread.sleep(100);
           }
@@ -1230,11 +1238,11 @@ public class ConnectionTable  {
     /**
      * the stub we are connecting to
      */
-    private final Stub id;
+    private final DistributedMember id;
     
     private final Thread connectingThread;
     
-    public PendingConnection(boolean preserveOrder, Stub id) {
+    public PendingConnection(boolean preserveOrder, DistributedMember id) {
       this.preserveOrder = preserveOrder;
       this.id = id;
       this.connectingThread = Thread.currentThread();
@@ -1279,10 +1287,9 @@ public class ConnectionTable  {
 
       boolean severeAlertIssued = false;
       boolean suspected = false;
-      InternalDistributedMember targetMember = null;
+      DistributedMember targetMember = null;
       if (ackSATimeout > 0) {
-        targetMember =
-          ((GMSMembershipManager)mgr).getMemberForStub(this.id, false);
+        targetMember = this.id;
       }
 
       for (;;) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
index 5cd426f..a954814 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
@@ -18,6 +18,7 @@
 package com.gemstone.gemfire.internal.tcp;
 
 import com.gemstone.gemfire.GemFireException;
+import com.gemstone.gemfire.distributed.DistributedMember;
 
 /**
  * MemberShunnedException may be thrown to prevent ack-ing a message
@@ -28,13 +29,13 @@ import com.gemstone.gemfire.GemFireException;
 public class MemberShunnedException extends GemFireException
 {
   private static final long serialVersionUID = -8453126202477831557L;
-  private Stub member;
+  private DistributedMember member;
   
   /**
    * constructor
    * @param member the member that was shunned
    */
-  public MemberShunnedException(Stub member) {
+  public MemberShunnedException(DistributedMember member) {
     super("");
     this.member = member;
   }
@@ -42,7 +43,7 @@ public class MemberShunnedException extends GemFireException
   /**
    * @return the member that was shunned
    */
-  public Stub getShunnedMember() {
+  public DistributedMember getShunnedMember() {
     return this.member;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
index fd495d9..cd711e7 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
@@ -16,6 +16,7 @@
  */
 package com.gemstone.gemfire.internal.tcp;
 
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.membership.*;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
@@ -34,7 +35,7 @@ import com.gemstone.gemfire.i18n.LogWriterI18n;
 public interface ServerDelegate {
 
   public void receive( DistributionMessage message, int bytesRead,
-                       Stub connId );
+                       DistributedMember connId );
 
   public LogWriterI18n getLogger();
 
@@ -42,5 +43,5 @@ public interface ServerDelegate {
    * Called when a possibly new member is detected by receiving a direct channel
    * message from him.
    */
-  public void newMemberConnected(InternalDistributedMember member, Stub id);
+  public void newMemberConnected(InternalDistributedMember member);
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4bf4557b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java
deleted file mode 100644
index 2e4b91b..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.tcp;
-
-import java.io.*;
-import java.net.*;
-
-import com.gemstone.gemfire.DataSerializable;
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-
-/** Stub represents an ip address and port.
-
-    @author Bruce Schuchardt
-    @since 2.0
-   
- */
-
-public class Stub implements Externalizable, DataSerializable
-{
-  private InetAddress inAddr;
-  private int port;
-  private int viewID;
-
-  public Stub() {
-    // public default needed for deserialization
-  }
-  
-  public Stub(InetAddress addr, int port, int vmViewID) {
-    viewID = vmViewID;
-    inAddr = addr;
-    this.port = port;
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (o == this) {
-      return true;
-    }
-    if (o instanceof Stub) {
-      Stub s = (Stub)o;
-      boolean result;
-      if (inAddr == null)
-        result = s.inAddr == null;
-      else
-        result = inAddr.equals(s.inAddr);
-      result = result && port == s.port;
-      if (this.viewID != 0 && s.viewID != 0) {
-        result = result && (this.viewID == s.viewID);
-      }
-      return result;
-    }
-    else {
-      return false;
-    }
-  }
-  
-  // hashCode equates to the address hashCode for fast connection lookup
-  @Override
-  public int hashCode() {
-    // do not use viewID in hashCode because it is changed after creating a stub
-    int result = 0;
-    // result += inAddr.hashCode(); // useless
-    result += port;
-    return result;
-  }
-  
-  public void setViewID(int viewID) {
-    this.viewID = viewID;
-  }
-  
-  public int getPort() {
-    return port;
-  }
-  
-  public int getViewID() {
-    return this.viewID;
-  }
-  
-  public InetAddress getInetAddress() {
-    return inAddr;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(80);
-    sb.append("tcp://");
-    if (inAddr == null)
-      sb.append("<null>");
-    else
-      sb.append(inAddr.toString());
-    if (this.viewID != 0) {
-      sb.append("<v"+this.viewID+">");
-    }
-    sb.append(":" + port);
-    return sb.toString();
-  }
-  
-  /**
-   * Writes the contents of this <code>Stub</code> to a
-   * <code>DataOutput</code>. 
-   *
-   * @since 3.0
-   */
-  public void toData(DataOutput out) 
-    throws IOException
-  {
-    DataSerializer.writeInetAddress(inAddr, out);
-    out.writeInt(port);
-    out.writeInt(viewID);
-  }
-  
-  /**
-   * Reads the contents of this <code>Stub</code> from a
-   * <code>DataOutput</code>. 
-   *
-   * @since 3.0
-   */
-  public void fromData(DataInput in)
-    throws IOException, ClassNotFoundException
-  {
-    inAddr = DataSerializer.readInetAddress(in);
-    this.port = in.readInt();
-    this.viewID = in.readInt();
-  }
-
-  /**
-   * static factory method
-   * @since 5.0.2
-   */
-  public static Stub createFromData(DataInput in)
-    throws IOException, ClassNotFoundException
-  {
-    Stub result = new Stub();
-    InternalDataSerializer.invokeFromData(result, in);
-    return result;
-  }
-  
-  public void writeExternal(ObjectOutput os) 
-    throws IOException
-  {
-    this.toData(os);
-  }
-  
-  public void readExternal(ObjectInput is)
-    throws IOException, ClassNotFoundException
-  {
-    this.fromData(is);
-  }
-}


[39/50] [abbrv] incubator-geode git commit: GEODE-338 FDDUnitTest. testFDSocketFixOnlyServers The issue this test was trying to reproduce no longer appears to be affecting the latest jvm Removing this test from the code base Created GEODE-679 to track re

Posted by je...@apache.org.
GEODE-338 FDDUnitTest. testFDSocketFixOnlyServers
The issue this test was trying to reproduce no longer appears to be affecting the latest jvm
Removing this test from the code base
Created GEODE-679 to track removal of private selector pool code


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

Branch: refs/heads/feature/GEODE-14
Commit: d40d8a70d084d963530cc132c9004cf39b1b3a76
Parents: e63ad64
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue Dec 15 14:58:58 2015 -0800
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Dec 15 15:11:24 2015 -0800

----------------------------------------------------------------------
 .../gemstone/gemfire/internal/FDDUnitTest.java  | 282 -------------------
 1 file changed, 282 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d40d8a70/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
deleted file mode 100644
index f1ce420..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/FDDUnitTest.java
+++ /dev/null
@@ -1,282 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal;
-
-import java.util.Properties;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import com.gemstone.gemfire.cache.CacheException;
-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.client.ClientCache;
-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.cache30.CacheSerializableRunnable;
-import com.gemstone.gemfire.cache30.CacheTestCase;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
-import com.gemstone.gemfire.internal.stats50.VMStats50;
-
-import dunit.Host;
-import dunit.SerializableCallable;
-import dunit.SerializableRunnable;
-import dunit.VM;
-
-/**
- * 
- * @author jhuynh
- *
- */
-public class FDDUnitTest extends CacheTestCase {
-
-  VM vm0;
-  VM vm1;
-  VM vm2;
-
-  
-  public FDDUnitTest(String name) {
-    super(name);
-  }
-
-  public void setUp() throws Exception {
-    super.setUp();
-    //getSystem();
-   
-    Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-    vm2 = host.getVM(2);
-  }
-  
-  public void tearDown2() throws Exception {
-    vm0.invoke(cleanup());
-    vm1.invoke(cleanup());
-    vm2.invoke(cleanup());
-    vm0.invoke(CacheServerTestUtil.class, "closeCache");
-    vm1.invoke(CacheServerTestUtil.class, "closeCache");
-    vm2.invoke(CacheServerTestUtil.class, "closeCache");
-  }
-
-  public void testEmpty() {
-    //Ticket #GEODE-338.  Disable the test for now and rewrite as a junit test.
-  }
-  
-  public void disable_testFDSocketFixOnlyServers() throws Exception {
-    String os = System.getProperty("os.name");
-    if (os != null) {
-      if (os.indexOf("Windows") != -1) {
-        System.out.println("This test is disabled on Windows");
-        //we are running this test on windows.  fd stats are not available in windows so let's
-        //just not run this test
-        return;
-      }
-    }
-    try {
-      StringBuffer incaseOfFailure = new StringBuffer();
-      final int[] port = AvailablePortHelper.getRandomAvailableTCPPorts(3);
-      int numThreads = 30;
-
-      startCacheServer(vm0, port[0]);
-      startCacheServer(vm1, port[1]);
-      startCacheServer(vm2, port[2]);
-
-      createRegion(vm0, "portfolios", RegionShortcut.PARTITION_REDUNDANT);
-      createRegion(vm1, "portfolios", RegionShortcut.PARTITION_REDUNDANT);
-      createRegion(vm2, "portfolios", RegionShortcut.PARTITION_REDUNDANT);
-
-      // run test without selector pooling
-      setUseSelectorPooling(vm0, false);
-      long startingFDs = checkFD(vm0);
-      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);
-      long startingFDsWithPooling = checkFD(vm0);
-      doPuts(vm0, numThreads, "portfolios");
-      long endFDsWithPooling = checkFD(vm0);
-      long numFDsWithPooling = endFDsWithPooling - startingFDsWithPooling;
-      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);
-      doPuts(vm0, numThreads, "portfolios");
-      endFDsWithPooling = checkFD(vm0);
-      numFDsWithPooling = endFDsWithPooling - startingFDsWithPooling;
-      // if you see these asserts failing, it could be that we are not using the
-      // selector pool
-      incaseOfFailure.append("SelectorPooling#2 startFDs: " + startingFDsWithPooling + " endFDs: " + endFDsWithPooling + " diff:" + numFDsWithPooling + " ");
-      assertTrue(incaseOfFailure.toString(), numFDsWithPooling < numFDs);
-
-    } finally {
-      setUseSelectorPooling(vm0, true);
-    }
-
-  }
-  
-  private void setUseSelectorPooling(VM vm, final boolean useSelectorPooling) {
-    vm.invoke(new SerializableRunnable("setting use selectorPooling to " + useSelectorPooling) {
-      public void run() {
-        SocketUtils.USE_SELECTOR_POOLING = useSelectorPooling;
-      }
-    });
-  }
-  private Long checkFD(VM vm) {
-    return (Long)vm.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        VMStatsContract stats = ((InternalDistributedSystem)system).getStatSampler().getVMStats();
-        VMStats50 vmstats = (VMStats50) stats;
-        return vmstats.getVMStats().get("fdsOpen").longValue();
-      }
-    });
-  }
-  
-  private void doPuts(VM vm, final int numThreads, final String regionName) {
-    vm.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        final Region region = getCache().getRegion(regionName);
-        if (region == null) {
-          throw new Exception("No Region found");
-        }
-        ExecutorService executor = Executors.newFixedThreadPool(numThreads);
-          for (int i = 0; i < numThreads; i++) {
-            executor.execute(new Runnable() {
-              public void run() {
-                  for (int i = 0; i < 10; i++) {
-                    String myValue = "string" + i;
-                    region.put("k" + i, myValue);
-                    try {
-                      Thread.sleep(75); 
-                    }
-                    catch (InterruptedException e) {
-                      Thread.currentThread().interrupt();
-                    }
-                    
-                  }
-              }
-            });
-          }
-        
-          executor.shutdown();
-       return executor.awaitTermination(90, TimeUnit.SECONDS);
-      }
-    });
-  }
-  
-  private String createGarbage(int valueIndex) {
-    StringBuffer[] randomStringArray = new StringBuffer[100];
-    for (int i = 0; i < randomStringArray.length; i++) {
-      randomStringArray[i] = new StringBuffer();
-      randomStringArray[i].append("value" + valueIndex + "," + Math.random());
-    }
-    return randomStringArray[(int)(Math.random() * randomStringArray.length)].toString();
-  }
- 
-  private void createRegion(VM vm, final String regionName, final RegionShortcut shortcut) {
-    vm.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        PartitionAttributesFactory paf = new PartitionAttributesFactory();
-        RegionFactory factory = getCache().createRegionFactory(shortcut)
-            .setPartitionAttributes(paf.create());
-        factory.create(regionName);
-        return null;
-      }
-    });
-  }
-  
-  private void createRegionOnClient(VM vm, final String regionName, final ClientRegionShortcut shortcut) {
-    vm.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        PartitionAttributesFactory paf = new PartitionAttributesFactory();
-        ClientRegionFactory factory = ((ClientCache)getCache()).createClientRegionFactory(shortcut);
-        factory.create(regionName);
-        return null;
-      }
-    });
-  }
-  
-  private void startCacheServer(VM server, final int port) throws Exception {
-    server.invoke(new SerializableCallable() {
-      public Object call() throws Exception {
-        //System.setProperty("IDLE_THREAD_TIMEOUT", "50");
-        disconnectFromDS();
-        
-        getSystem(getServerProperties());
-        
-        GemFireCacheImpl cache = (GemFireCacheImpl)getCache();
-        
-        CacheServer cacheServer = cache.addCacheServer();
-        cacheServer.setPort(port);
-        cacheServer.start();  
-        return null;
-      }
-    });
-  }
-  
-  private void startClient(VM client, final VM server, final int port) {
-    client.invoke(new CacheSerializableRunnable("Start client") {
-      public void run2() throws CacheException {
-        //System.setProperty("IDLE_THREAD_TIMEOUT", "50");
-        Properties props = getClientProps();
-        getSystem(props);
-        
-        final ClientCacheFactory ccf = new ClientCacheFactory(props);
-        ccf.addPoolServer(getServerHostName(server.getHost()), port);
-        ccf.setPoolSubscriptionEnabled(true);
-        
-        ClientCache cache = (ClientCache)getClientCache(ccf);
-      }
-    });
-  }
-  
-  private Runnable cleanup() {
-    return new SerializableRunnable() {
-      public void run() {
-        //System.setProperty("IDLE_THREAD_TIMEOUT", "30000*60");
-      }
-    };
-  }
- 
-  protected Properties getClientProps() {
-    Properties p = new Properties();
-    p.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
-    p.setProperty(DistributionConfig.LOCATORS_NAME, "");
-    p.setProperty(DistributionConfig.CONSERVE_SOCKETS_NAME, "false");
-    return p;
-  }
-
-  protected Properties getServerProperties() {
-    Properties p = new Properties();
-    p.setProperty(DistributionConfig.LOCATORS_NAME, "localhost["+getDUnitLocatorPort()+"]");
-    p.setProperty(DistributionConfig.CONSERVE_SOCKETS_NAME, "false");
-    //p.setProperty(DistributionConfig.SOCKET_LEASE_TIME_NAME, "120000");
-    return p;
-  }
- 
-
-}


[43/50] [abbrv] incubator-geode git commit: Merge branch 'feature/GEODE-568' into develop

Posted by je...@apache.org.
Merge branch 'feature/GEODE-568' into develop

Closes #59


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

Branch: refs/heads/feature/GEODE-14
Commit: 0e366b0263eefc685d0e4f04122ae165b5088afe
Parents: f2a852d d58588a
Author: Jens Deppe <jd...@pivotal.io>
Authored: Thu Dec 17 10:34:57 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Thu Dec 17 10:34:57 2015 -0800

----------------------------------------------------------------------
 .../gemfire/management/CacheManagementDUnitTest.java     | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[30/50] [abbrv] incubator-geode git commit: recommitting 5b35e43f93bfbf6d62eadf7979eb3a8b7f59b77e and fixing compilation error

Posted by je...@apache.org.
recommitting 5b35e43f93bfbf6d62eadf7979eb3a8b7f59b77e and fixing compilation error

GMSMembershipManager had unsaved changes in my Eclipse browser.  Not sure
why my pre-commit gradle build didn't catch that problem - sorry everyone.


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

Branch: refs/heads/feature/GEODE-14
Commit: 9efb5867db9d6fc02cccf78fb8da051e86ab6887
Parents: 4bf4557
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Tue Dec 15 08:31:53 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Tue Dec 15 08:31:53 2015 -0800

----------------------------------------------------------------------
 .../internal/membership/gms/mgr/GMSMembershipManager.java        | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9efb5867/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index 7be0a3a..56643ad 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -2213,7 +2213,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
   /* non-thread-owned serial channels and high priority channels are not
    * included
    */
-  public Map getChannelStates(DistributedMember member, boolean includeMulticast) {
+  public Map getMessageState(DistributedMember member, boolean includeMulticast) {
     Map result = new HashMap();
     DirectChannel dc = directChannel;
     if (dc != null) {
@@ -2229,7 +2229,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     if (Thread.interrupted()) throw new InterruptedException();
     DirectChannel dc = directChannel;
     if (dc != null) {
-      dc.waitForChannelState(otherMember, channelState);
+      dc.waitForChannelState(otherMember, state);
     }
     services.getMessenger().waitForMessageState((InternalDistributedMember)otherMember, state);
   }


[08/50] [abbrv] incubator-geode git commit: Removing TCPConduit's Stub ID class

Posted by je...@apache.org.
Removing TCPConduit's Stub ID class

This removes the Stub identifier class from TCPConduit.  This simplifies the
code since the DistributedMember IDs are propagated to all of the methods
that require identifiers and these IDs have all of the information Stubs
had.

The MembershipManager is also simplified since it doesn't have to keep
complicated mappings between Stubs and DistributedMembers.


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

Branch: refs/heads/feature/GEODE-14
Commit: 5b35e43f93bfbf6d62eadf7979eb3a8b7f59b77e
Parents: 2f0c7fc
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Dec 11 15:09:52 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Dec 11 15:38:03 2015 -0800

----------------------------------------------------------------------
 .../internal/DistributionManager.java           |   8 -
 .../distributed/internal/StartupMessage.java    |   1 -
 .../internal/direct/DirectChannel.java          |  93 +------
 .../internal/direct/MissingStubException.java   |  37 ---
 .../internal/direct/ShunnedMemberException.java |  34 +++
 .../internal/membership/MembershipManager.java  |  29 +-
 .../gms/mgr/GMSMembershipManager.java           | 197 ++-----------
 .../internal/i18n/ParentLocalizedStrings.java   |   6 +-
 .../gemfire/internal/tcp/Connection.java        | 117 ++++----
 .../gemfire/internal/tcp/ConnectionTable.java   |  91 +++---
 .../internal/tcp/MemberShunnedException.java    |   7 +-
 .../gemfire/internal/tcp/ServerDelegate.java    |   5 +-
 .../com/gemstone/gemfire/internal/tcp/Stub.java | 164 -----------
 .../gemfire/internal/tcp/TCPConduit.java        | 274 +++----------------
 .../internal/DistributionManagerDUnitTest.java  |   6 +-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |  31 +--
 .../internal/tcp/ConnectionJUnitTest.java       |   3 +-
 17 files changed, 233 insertions(+), 870 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
index 964845c..e3c342a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionManager.java
@@ -91,7 +91,6 @@ import com.gemstone.gemfire.internal.sequencelog.MembershipLogger;
 import com.gemstone.gemfire.internal.tcp.Connection;
 import com.gemstone.gemfire.internal.tcp.ConnectionTable;
 import com.gemstone.gemfire.internal.tcp.ReenteredConnectException;
-import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantLock;
 
 /**
@@ -2715,13 +2714,6 @@ public class DistributionManager
       return false; // no peers, we are alone.
     }
 
-    // ensure we have stubs for everyone else
-    Iterator it = allOthers.iterator();
-    while (it.hasNext()) {
-      InternalDistributedMember member = (InternalDistributedMember)it.next();
-      membershipManager.getStubForMember(member);
-    }
-
     try {
       ok = op.sendStartupMessage(allOthers, STARTUP_TIMEOUT, equivs,
           redundancyZone, enforceUniqueZone());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
index 96f8b60..01f8c62 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/StartupMessage.java
@@ -37,7 +37,6 @@ import com.gemstone.gemfire.internal.InternalDataSerializer.SerializerAttributes
 import com.gemstone.gemfire.internal.InternalInstantiator.InstantiatorAttributesHolder;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.internal.tcp.Stub;
 
 /**
  * A message that is sent to all other distribution manager when

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
index 14ff923..d4df3bf 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
@@ -38,6 +38,7 @@ import com.gemstone.gemfire.InternalGemFireException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.ToDataException;
 import com.gemstone.gemfire.cache.TimeoutException;
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
@@ -64,7 +65,6 @@ import com.gemstone.gemfire.internal.tcp.Connection;
 import com.gemstone.gemfire.internal.tcp.ConnectionException;
 import com.gemstone.gemfire.internal.tcp.MemberShunnedException;
 import com.gemstone.gemfire.internal.tcp.MsgStreamer;
-import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.tcp.TCPConduit;
 import com.gemstone.gemfire.internal.util.Breadcrumbs;
 import com.gemstone.gemfire.internal.util.concurrent.ReentrantSemaphore;
@@ -115,13 +115,6 @@ public class DirectChannel {
     }
     
     /**
-     * Returns the endpoint ID for the direct channel
-     */
-    public Stub getLocalStub() {
-      return conduit.getId();
-    }
-    
-    /**
      * when the initial number of members is known, this method is invoked
      * to ensure that connections to those members can be established in a
      * reasonable amount of time.  See bug 39848 
@@ -181,7 +174,7 @@ public class DirectChannel {
         this.groupOrderedSenderSem = new ReentrantSemaphore(MAX_GROUP_SENDERS);
         this.groupUnorderedSenderSem = new ReentrantSemaphore(MAX_GROUP_SENDERS);
         logger.info(LocalizedMessage.create(
-            LocalizedStrings.DirectChannel_GEMFIRE_P2P_LISTENER_STARTED_ON__0, conduit.getId()));
+            LocalizedStrings.DirectChannel_GEMFIRE_P2P_LISTENER_STARTED_ON__0, conduit.getLocalAddr()));
 
       }
       catch (ConnectionException ce) {
@@ -192,48 +185,6 @@ public class DirectChannel {
     }
 
  
-//   /**
-//    * 
-//    * @param addr destination for the message
-//    * @param stubMap map containing all the stubs
-//    * @param msg the original message
-//    * @param msgBuf the serialized message
-//    * @param directAck true if we need an ack
-//    * @param processorType the type (serialized, etc.)
-//    * @return if directAck, the Connection that needs the acknowledgment
-//    * @throws MissingStubException if we do not have a Stub for the recipient
-//    * @throws IOException if the message could not be sent
-//    */
-//   private Connection attemptSingleSend(MembershipManager mgr,
-//       InternalDistributedMember addr,
-//       DistributionMessage msg, ByteBuffer msgBuf,
-//       boolean directAck, int processorType)
-//       throws MissingStubException, IOException
-//   {
-//     if (!msg.deliverToSender() && localAddr.equals(addr))
-//       return null;
-
-//     if (addr == null)
-//       return null;
-//     Stub dest = mgr.getStubForMember(addr);
-//     if (dest == null) {
-//       // This should only happen if the member is no longer in the view.
-//       Assert.assertTrue(!mgr.memberExists(addr));
-//       throw new MissingStubException("No stub");
-//     }
-//     try {
-//       msgBuf.position(0); // fix for bug#30680
-//       Connection con = conduit.sendSync(dest, msgBuf, processorType, msg);
-//       if (directAck)
-//         return con;
-//       else
-//         return null;
-//     }
-//     catch(IOException t) {
-//       throw t;
-//       }
-//   }
-
   /**
    * Return how many concurrent operations should be allowed by default.
    * since 6.6, this has been raised to Integer.MAX value from the number
@@ -639,22 +590,13 @@ public class DirectChannel {
         continue;
       }
 
-      Stub stub = mgr.getStubForMember(destination);
-      if (stub == null) {
+      if (!mgr.memberExists(destination) || mgr.shutdownInProgress() || mgr.isShunned(destination)) {
         // This should only happen if the member is no longer in the view.
         if (logger.isTraceEnabled(LogMarker.DM)) {
-          logger.trace(LogMarker.DM, "No Stub for {}", destination);
+          logger.trace(LogMarker.DM, "Not a member: {}", destination);
         }
-        // The only time getStubForMember returns null is if we are
-        // shunning that member or we are shutting down.
-        // So the following assertion is wrong:
-        //Assert.assertTrue(!mgr.memberExists(destination));
-        // instead we should:
-        // Assert.assertTrue(mgr.shutdownInProgress() || mgr.isShunned(destination));
-        //but this is not worth doing and isShunned is not public.
-        // SO the assert has been deadcoded.
         if (ce == null) ce = new ConnectExceptions();
-        ce.addFailure(destination, new MissingStubException(LocalizedStrings.DirectChannel_NO_STUB_0.toLocalizedString()));
+        ce.addFailure(destination, new ShunnedMemberException(LocalizedStrings.DirectChannel_SHUNNING_0.toLocalizedString()));
       }
       else {
         try {
@@ -662,8 +604,8 @@ public class DirectChannel {
           if (ackTimeout > 0) {
             startTime = System.currentTimeMillis();
           }
-          Connection con = conduit.getConnection(destination, stub,
-              preserveOrder, retry, startTime, ackTimeout, ackSDTimeout);
+          Connection con = conduit.getConnection(destination, preserveOrder,
+              retry, startTime, ackTimeout, ackSDTimeout);
           
           con.setInUse(true, startTime, 0, 0, null); // fix for bug#37657
           cons.add(con);
@@ -823,7 +765,7 @@ public class DirectChannel {
   }
 
   
-  public void receive(DistributionMessage msg, int bytesRead, Stub connId) {
+  public void receive(DistributionMessage msg, int bytesRead) {
     if (disconnected) {
       return;
     }
@@ -844,10 +786,6 @@ public class DirectChannel {
     }
   }
 
-//  public void newMemberConnected(InternalDistributedMember member, Stub id) {
-//    receiver.newMemberConnected(member, id);
-//  }
-
   public InternalDistributedMember getLocalAddress() {
     return this.localAddr;
   }
@@ -930,13 +868,6 @@ public class DirectChannel {
     }
   }
   
-  /** Create a TCPConduit stub from a JGroups InternalDistributedMember */
-  public Stub createConduitStub(InternalDistributedMember addr) {
-    int port = addr.getDirectChannelPort();
-    Stub stub = new Stub(addr.getInetAddress(), port, addr.getVmViewId());
-    return stub;
-  }
-  
   public void closeEndpoint(InternalDistributedMember member, String reason) {
     closeEndpoint(member, reason, true);
   }
@@ -948,7 +879,7 @@ public class DirectChannel {
   public void closeEndpoint(InternalDistributedMember member, String reason, boolean notifyDisconnect) {
     TCPConduit tc = this.conduit;
     if (tc != null) {
-      tc.removeEndpoint(createConduitStub(member), reason, notifyDisconnect);
+      tc.removeEndpoint(member, reason, notifyDisconnect);
     }
   }
 
@@ -962,7 +893,7 @@ public class DirectChannel {
    *    the map to add the state to
    * @since 5.1
    */
-  public void getChannelStates(Stub member, Map result)
+  public void getChannelStates(DistributedMember member, Map result)
   {
     TCPConduit tc = this.conduit;
     if (tc != null) {
@@ -974,7 +905,7 @@ public class DirectChannel {
    * wait for the given connections to process the number of messages
    * associated with the connection in the given map
    */
-  public void waitForChannelState(Stub member, Map channelState)
+  public void waitForChannelState(DistributedMember member, Map channelState)
     throws InterruptedException
   {
     if (Thread.interrupted()) throw new InterruptedException();
@@ -987,7 +918,7 @@ public class DirectChannel {
   /**
    * returns true if there are still receiver threads for the given member
    */
-  public boolean hasReceiversFor(Stub mbr) {
+  public boolean hasReceiversFor(DistributedMember mbr) {
     return this.conduit.hasReceiversFor(mbr);
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java
deleted file mode 100644
index 49b4486..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/MissingStubException.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.distributed.internal.direct;
-
-import com.gemstone.gemfire.GemFireCheckedException;
-
-/**
- * Exception thrown when the TCPConduit is unable to acquire a stub
- * for the given recipient.
- * 
- * @author jpenney
- *
- */
-public class MissingStubException extends GemFireCheckedException
-{
-
-  private static final long serialVersionUID = -6455664684151074915L;
-
-  public MissingStubException(String msg) {
-    super(msg);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java
new file mode 100644
index 0000000..59db762
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/ShunnedMemberException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed.internal.direct;
+
+import com.gemstone.gemfire.GemFireCheckedException;
+
+/**
+ * Exception thrown when a member is no longer in the distributed system
+ * 
+ */
+public class ShunnedMemberException extends GemFireCheckedException
+{
+
+  private static final long serialVersionUID = -6455664684151074915L;
+
+  public ShunnedMemberException(String msg) {
+    super(msg);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
index a46680b..7416efa 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
@@ -27,7 +27,6 @@ import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
-import com.gemstone.gemfire.internal.tcp.Stub;
 
 /**
  * A MembershipManager is responsible for reporting a MemberView, as well as
@@ -74,7 +73,7 @@ public interface MembershipManager {
    * @param m the member
    * @return true if it still exists
    */
-  public boolean memberExists(InternalDistributedMember m);
+  public boolean memberExists(DistributedMember m);
   
   /**
    * Is this manager still connected?  If it has not been initialized, this
@@ -143,25 +142,6 @@ public interface MembershipManager {
   throws NotSerializableException;
   
   /**
-   * Return a {@link Stub} referring to the given member.  A <em>null</em> may
-   * be returned if the system is not employing stubs for communication.
-   * 
-   * @param m the member
-   * @return the stub
-   */
-  public Stub getStubForMember(InternalDistributedMember m);
-  
-  /**
-   * Return a {@link InternalDistributedMember} associated with the given Stub.  This
-   * method may return a null if Stubs are not being used.
-   * @param s Stub to look up
-   * @param validated true if member must be in the current view
-   * @return the member associated with the given stub, if any
-   */
-  public InternalDistributedMember getMemberForStub(Stub s, boolean validated);
-  
-  
-  /**
    * Indicates to the membership manager that the system is shutting down.
    * Typically speaking, this means that new connection attempts are to be
    * ignored and disconnect failures are to be (more) tolerated.
@@ -286,7 +266,7 @@ public interface MembershipManager {
    */
   public void warnShun(DistributedMember mbr);
   
-  public boolean addSurpriseMember(DistributedMember mbr, Stub stub);
+  public boolean addSurpriseMember(DistributedMember mbr);
   
   /** if a StartupMessage is going to reject a new member, this should be used
    * to make sure we don't keep that member on as a "surprise member"
@@ -307,6 +287,11 @@ public interface MembershipManager {
    * @return true if the member is a surprise member
    */
   public boolean isSurpriseMember(DistributedMember m);
+  
+  /**
+   * Returns true if the member is being shunned
+   */
+  public boolean isShunned(DistributedMember m);
 
   /**
    * Forces use of UDP for communications in the current thread.  UDP is

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index 0b7a544..7be0a3a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -94,7 +94,6 @@ import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 import com.gemstone.gemfire.internal.shared.StringPrintWriter;
 import com.gemstone.gemfire.internal.tcp.ConnectExceptions;
 import com.gemstone.gemfire.internal.tcp.MemberShunnedException;
-import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.internal.util.Breadcrumbs;
 
 public class GMSMembershipManager implements MembershipManager, Manager
@@ -156,7 +155,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     boolean crashed;
     String reason;
     DistributionMessage dmsg;
-    Stub stub;
     NetView gmsView;
     
     @Override
@@ -165,7 +163,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       sb.append("kind=");
       switch (kind) {
       case SURPRISE_CONNECT:
-        sb.append("connect; member = <" + member + ">; stub = " + stub);
+        sb.append("connect; member = <" + member + ">");
         break;
       case VIEW:
         String text = gmsView.toString();
@@ -184,12 +182,10 @@ public class GMSMembershipManager implements MembershipManager, Manager
     /**
      * Create a surprise connect event
      * @param member the member connecting
-     * @param id the stub
      */
-    StartupEvent(final InternalDistributedMember member, final Stub id) {
+    StartupEvent(final InternalDistributedMember member) {
       this.kind = SURPRISE_CONNECT;
       this.member = member;
-      this.stub = id;
     }
     /**
      * Indicate if this is a surprise connect event
@@ -282,24 +278,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
   volatile boolean hasJoined;
   
   /**
-   * a map keyed on InternalDistributedMember, values are Stubs that represent direct
-   * channels to other systems
-   * 
-   * Accesses must be under the read or write lock of {@link #latestViewLock}.
-   */
-  protected final Map<InternalDistributedMember, Stub> memberToStubMap = 
-      new ConcurrentHashMap<InternalDistributedMember, Stub>();
-
-  /**
-   * a map of direct channels (Stub) to InternalDistributedMember. key instanceof Stub
-   * value instanceof InternalDistributedMember
-   * 
-   * Accesses must be under the read or write lock of {@link #latestViewLock}.
-   */
-  protected final Map<Stub, InternalDistributedMember> stubToMemberMap = 
-      new ConcurrentHashMap<Stub, InternalDistributedMember>();
-  
-  /**
    * Members of the distributed system that we believe have shut down.
    * Keys are instances of {@link InternalDistributedMember}, values are 
    * Longs indicating the time this member was shunned.
@@ -547,12 +525,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
           }
         }
 
-        // fix for bug #42006, lingering old identity
-        Object oldStub = this.memberToStubMap.remove(m);
-        if (oldStub != null) {
-          this.stubToMemberMap.remove(oldStub);
-        }
-
         if (shutdownInProgress()) {
           addShunnedMember(m);
           continue; // no additions processed after shutdown begins
@@ -806,9 +778,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     
     if (directChannel != null) {
       directChannel.setLocalAddr(address);
-      Stub stub = directChannel.getLocalStub();
-      memberToStubMap.put(address, stub);
-      stubToMemberMap.put(stub, address);
     }
 
     this.hasJoined = true;
@@ -905,17 +874,15 @@ public class GMSMembershipManager implements MembershipManager, Manager
   /**
    * Process a surprise connect event, or place it on the startup queue.
    * @param member the member
-   * @param stub its stub
    */
   protected void handleOrDeferSurpriseConnect(InternalDistributedMember member) {
-    Stub stub = new Stub(member.getInetAddress(), member.getDirectChannelPort(), member.getVmViewId());
     synchronized (startupLock) {
       if (!processingEvents) {
-        startupMessages.add(new StartupEvent(member, stub));
+        startupMessages.add(new StartupEvent(member));
         return;
       }
     }
-    processSurpriseConnect(member, stub);
+    processSurpriseConnect(member);
   }
   
   public void startupMessageFailed(DistributedMember mbr, String failureMessage) {
@@ -941,12 +908,9 @@ public class GMSMembershipManager implements MembershipManager, Manager
    * been added, simply returns; else adds the member.
    * 
    * @param dm the member joining
-   * @param stub the member's stub
    */
-  public boolean addSurpriseMember(DistributedMember dm, 
-      Stub stub) {
+  public boolean addSurpriseMember(DistributedMember dm) {
     final InternalDistributedMember member = (InternalDistributedMember)dm;
-    Stub s = null;
     boolean warn = false;
     
     latestViewLock.writeLock().lock();
@@ -1009,16 +973,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
           startCleanupTimer();
         } // cleanupTimer == null
 
-        // fix for bug #42006, lingering old identity
-        Object oldStub = this.memberToStubMap.remove(member);
-        if (oldStub != null) {
-          this.stubToMemberMap.remove(oldStub);
-        }
-
-        s = stub == null ? getStubForMember(member) : stub;
-        // Make sure that channel information is consistent
-        addChannel(member, s);
-
         // Ensure that the member is accounted for in the view
         // Conjure up a new view including the new member. This is necessary
         // because we are about to tell the listener about a new member, so
@@ -1154,7 +1108,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
 
         // If it's a new sender, wait our turn, generate the event
         if (isNew) {
-          shunned = !addSurpriseMember(m, getStubForMember(m));
+          shunned = !addSurpriseMember(m);
         } // isNew
       }
 
@@ -1166,7 +1120,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     if (shunned) { // bug #41538 - shun notification must be outside synchronization to avoid hanging
       warnShun(m);
       logger.info("Membership: Ignoring message from shunned member <{}>:{}", m, msg);
-      throw new MemberShunnedException(getStubForMember(m));
+      throw new MemberShunnedException(m);
     }
     
     listener.messageReceived(msg);
@@ -1248,13 +1202,11 @@ public class GMSMembershipManager implements MembershipManager, Manager
    * grabbed a stable view if this is really a new member.
    * 
    * @param member
-   * @param stub
    */
   private void processSurpriseConnect(
-      InternalDistributedMember member, 
-      Stub stub) 
+      InternalDistributedMember member) 
   {
-    addSurpriseMember(member, stub);
+    addSurpriseMember(member);
   }
   
   /**
@@ -1276,7 +1228,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       processView(o.gmsView.getViewId(), o.gmsView);
     }
     else if (o.isSurpriseConnect()) { // connect
-      processSurpriseConnect(o.member, o.stub);
+      processSurpriseConnect(o.member);
     }
     
     else // sanity
@@ -1450,7 +1402,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     }
   }
 
-  public boolean memberExists(InternalDistributedMember m) {
+  public boolean memberExists(DistributedMember m) {
     latestViewLock.readLock().lock();
     NetView v = latestView;
     latestViewLock.readLock().unlock();
@@ -1525,12 +1477,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
       directChannel.emergencyClose();
     }
     
-    // could we guarantee not to allocate objects?  We're using Darrel's 
-    // factory, so it's possible that an unsafe implementation could be
-    // introduced here.
-//    stubToMemberMap.clear();
-//    memberToStubMap.clear();
-    
     if (DEBUG) {
       System.err.println("DEBUG: done closing GroupMembershipService");
     }
@@ -1767,7 +1713,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
       allDestinations = true;
       latestViewLock.writeLock().lock();
       try {
-        Set keySet = memberToStubMap.keySet();
+        List<InternalDistributedMember> keySet = latestView.getMembers();
         keys = new InternalDistributedMember[keySet.size()];
         keys = (InternalDistributedMember[])keySet.toArray(keys);
       } finally {
@@ -2020,80 +1966,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     // not currently supported by this manager
   }
   
-  /**
-   * Get or create stub for given member
-   */
-  public Stub getStubForMember(InternalDistributedMember m)
-  {
-    if (shutdownInProgress) {
-      throw new DistributedSystemDisconnectedException(LocalizedStrings.GroupMembershipService_DISTRIBUTEDSYSTEM_IS_SHUTTING_DOWN.toLocalizedString(), services.getShutdownCause());
-    }
-
-    if (services.getConfig().getDistributionConfig().getDisableTcp()) {
-      return new Stub(m.getInetAddress(), m.getPort(), m.getVmViewId());
-    }
-    
-    // Return existing one if it is already in place
-    Stub result;
-    result = (Stub)memberToStubMap.get(m);
-    if (result != null)
-      return result;
-
-    latestViewLock.writeLock().lock();
-    try {
-      // Do all of this work in a critical region to prevent
-      // members from slipping in during shutdown
-      if (shutdownInProgress())
-        return null; // don't try to create a stub during shutdown
-      if (isShunned(m))
-        return null; // don't let zombies come back to life
-      
-      // OK, create one.  Update the table to reflect the creation.
-      result = directChannel.createConduitStub(m);
-      addChannel(m, result);
-    } finally {
-      latestViewLock.writeLock().unlock();
-    }
-   return result;
-  }
-
-  public InternalDistributedMember getMemberForStub(Stub s, boolean validated)
-  {
-    latestViewLock.writeLock().lock();
-    try {
-      if (shutdownInProgress) {
-        throw new DistributedSystemDisconnectedException(LocalizedStrings.GroupMembershipService_DISTRIBUTEDSYSTEM_IS_SHUTTING_DOWN.toLocalizedString(), services.getShutdownCause());
-      }
-      InternalDistributedMember result = (InternalDistributedMember)
-          stubToMemberMap.get(s);
-      if (result != null) {
-        if (validated && !this.latestView.contains(result)) {
-          // Do not return this member unless it is in the current view.
-          if (!surpriseMembers.containsKey(result)) {
-            // if not a surprise member, this stub is lingering and should be removed
-            stubToMemberMap.remove(s);
-            memberToStubMap.remove(result);
-          }
-          result = null;
-          // fall through to see if there is a newer member using the same direct port
-        }
-      }
-      if (result == null) {
-        // it may have not been added to the stub->idm map yet, so check the current view
-        for (InternalDistributedMember idm: latestView.getMembers()) {
-          if (GMSUtil.compareAddresses(idm.getInetAddress(), s.getInetAddress()) == 0
-              && idm.getDirectChannelPort() == s.getPort()) {
-            addChannel(idm, s);
-            return idm;
-          }
-        }
-      }
-      return result;
-    } finally {
-      latestViewLock.writeLock().unlock();
-    }
-  }
-
   public void setShutdown()
   {
     latestViewLock.writeLock().lock();
@@ -2109,24 +1981,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     return shutdownInProgress || (dm != null && dm.shutdownInProgress());
   }
   
-  /**
-   * Add a mapping from the given member to the given stub. Must
-   * be called with {@link #latestViewLock} held.
-   * 
-   * @param member
-   * @param theChannel
-   */
-  protected void addChannel(InternalDistributedMember member, Stub theChannel)
-  {
-    if (theChannel != null) {
-      // Don't overwrite existing stub information with a null
-      this.memberToStubMap.put(member, theChannel);
-
-      // Can't create reverse mapping if the stub is null
-      this.stubToMemberMap.put(theChannel, member);
-    }
-  }
-
 
   /**
    * Clean up and create consistent new view with member removed.
@@ -2137,12 +1991,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
   protected void destroyMember(final InternalDistributedMember member,
       boolean crashed, final String reason) {
     
-    // Clean up the maps
-    Stub theChannel = (Stub)memberToStubMap.remove(member);
-    if (theChannel != null) {
-      this.stubToMemberMap.remove(theChannel);
-    }
-    
     // Make sure it is removed from the view
     latestViewLock.writeLock().lock();
     try {
@@ -2365,12 +2213,11 @@ public class GMSMembershipManager implements MembershipManager, Manager
   /* non-thread-owned serial channels and high priority channels are not
    * included
    */
-  public Map getMessageState(DistributedMember member, boolean includeMulticast) {
+  public Map getChannelStates(DistributedMember member, boolean includeMulticast) {
     Map result = new HashMap();
-    Stub stub = (Stub)memberToStubMap.get(member);
     DirectChannel dc = directChannel;
-    if (stub != null && dc != null) {
-      dc.getChannelStates(stub, result);
+    if (dc != null) {
+      dc.getChannelStates(member, result);
     }
     services.getMessenger().getMessageState((InternalDistributedMember)member, result, includeMulticast);
     return result;
@@ -2381,15 +2228,8 @@ public class GMSMembershipManager implements MembershipManager, Manager
   {
     if (Thread.interrupted()) throw new InterruptedException();
     DirectChannel dc = directChannel;
-    Stub stub;
-    latestViewLock.writeLock().lock();
-    try {
-      stub = (Stub)memberToStubMap.get(otherMember);
-    } finally {
-      latestViewLock.writeLock().unlock();
-    }
-    if (dc != null && stub != null) {
-      dc.waitForChannelState(stub, state);
+    if (dc != null) {
+      dc.waitForChannelState(otherMember, channelState);
     }
     services.getMessenger().waitForMessageState((InternalDistributedMember)otherMember, state);
   }
@@ -2405,7 +2245,6 @@ public class GMSMembershipManager implements MembershipManager, Manager
     boolean result = false;
     DirectChannel dc = directChannel;
     InternalDistributedMember idm = (InternalDistributedMember)mbr;
-    Stub stub = new Stub(idm.getInetAddress(), idm.getPort(), idm.getVmViewId());
     int memberTimeout = this.services.getConfig().getDistributionConfig().getMemberTimeout();
     long pauseTime = (memberTimeout < 1000) ? 100 : memberTimeout / 10;
     boolean wait;
@@ -2413,7 +2252,7 @@ public class GMSMembershipManager implements MembershipManager, Manager
     do {
       wait = false;
       if (dc != null) {
-        if (dc.hasReceiversFor(stub)) {
+        if (dc.hasReceiversFor(idm)) {
           wait = true;
         }
         if (wait && logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/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 7bb97b9..780fe18 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
@@ -1109,7 +1109,7 @@ class ParentLocalizedStrings {
   public static final StringId TCPConduit_ENDING_RECONNECT_ATTEMPT_BECAUSE_0_HAS_DISAPPEARED = new StringId(2086, "Ending reconnect attempt because {0} has disappeared.");
   public static final StringId TCPConduit_ENDING_RECONNECT_ATTEMPT_TO_0_BECAUSE_SHUTDOWN_HAS_STARTED = new StringId(2087, "Ending reconnect attempt to {0} because shutdown has started.");
   public static final StringId TCPConduit_ERROR_SENDING_MESSAGE_TO_0_WILL_REATTEMPT_1 = new StringId(2088, "Error sending message to {0} (will reattempt): {1}");
-  public static final StringId TCPConduit_EXCEPTION_CREATING_SERVERSOCKET = new StringId(2089, "While creating ServerSocket and Stub on port {0} with address {1}");
+  public static final StringId TCPConduit_EXCEPTION_CREATING_SERVERSOCKET = new StringId(2089, "While creating ServerSocket on port {0} with address {1}");
   public static final StringId TCPConduit_EXCEPTION_PARSING_P2PIDLECONNECTIONTIMEOUT = new StringId(2090, "exception parsing p2p.idleConnectionTimeout");
   public static final StringId TCPConduit_EXCEPTION_PARSING_P2PTCPBUFFERSIZE = new StringId(2091, "exception parsing p2p.tcpBufferSize");
   public static final StringId TCPConduit_FAILED_TO_ACCEPT_CONNECTION_FROM_0_BECAUSE_1 = new StringId(2092, "Failed to accept connection from {0} because {1}");
@@ -1444,7 +1444,7 @@ class ParentLocalizedStrings {
   public static final StringId Connection_DETECTED_WRONG_VERSION_OF_GEMFIRE_PRODUCT_DURING_HANDSHAKE_EXPECTED_0_BUT_FOUND_1 = new StringId(2432, "Detected wrong version of GemFire product during handshake. Expected  {0}  but found  {1}");
   public static final StringId Connection_FORCED_DISCONNECT_SENT_TO_0 = new StringId(2433, "Forced disconnect sent to  {0}");
   public static final StringId Connection_HANDSHAKE_FAILED = new StringId(2434, "Handshake failed");
-  public static final StringId Connection_MEMBER_FOR_STUB_0_LEFT_THE_GROUP = new StringId(2435, "Member for stub  {0}  left the group");
+  public static final StringId Connection_MEMBER_LEFT_THE_GROUP = new StringId(2435, "Member {0}  left the group");
   public static final StringId Connection_NOT_CONNECTED_TO_0 = new StringId(2436, "Not connected to  {0}");
   public static final StringId Connection_NULL_CONNECTIONTABLE = new StringId(2437, "Null ConnectionTable");
   public static final StringId Connection_SOCKET_HAS_BEEN_CLOSED = new StringId(2438, "socket has been closed");
@@ -1542,7 +1542,7 @@ class ParentLocalizedStrings {
   public static final StringId DefaultQuery_WHEN_QUERYING_A_PARTITIONED_REGION_THE_PROJECTIONS_MUST_NOT_REFERENCE_ANY_REGIONS = new StringId(2530, "When querying a Partitioned Region, the projections must not reference any regions");
   public static final StringId DestroyMessage_FAILED_SENDING_0 = new StringId(2531, "Failed sending < {0} >");
   public static final StringId DirectChannel_COMMUNICATIONS_DISCONNECTED = new StringId(2532, "communications disconnected");
-  public static final StringId DirectChannel_NO_STUB_0 = new StringId(2533, "No stub {0}");
+  public static final StringId DirectChannel_SHUNNING_0 = new StringId(2533, "Member is being shunned: {0}");
   public static final StringId DirectChannel_UNKNOWN_ERROR_SERIALIZING_MESSAGE = new StringId(2534, "Unknown error serializing message");
   public static final StringId DiskEntry_AN_IOEXCEPTION_WAS_THROWN_WHILE_SERIALIZING = new StringId(2535, "An IOException was thrown while serializing.");
   public static final StringId DiskEntry_DISK_REGION_IS_NULL = new StringId(2536, "Disk region is null");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
index f918812..74660da 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Connection.java
@@ -50,6 +50,7 @@ import org.apache.logging.log4j.Logger;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.ConflationKey;
 import com.gemstone.gemfire.distributed.internal.DM;
@@ -72,7 +73,6 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.ByteArrayDataInput;
 import com.gemstone.gemfire.internal.DSFIDFactory;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
-import com.gemstone.gemfire.internal.SocketCloser;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.SocketUtils;
 import com.gemstone.gemfire.internal.SystemTimer;
@@ -222,11 +222,6 @@ public class Connection implements Runnable {
   /** the ID string of the conduit (for logging) */
   String conduitIdStr;
 
-  /** remoteId identifies the remote conduit's listener.  It does NOT
-     identify the "port" that this connection's socket is attached
-     to, which is a different thing altogether */
-  Stub remoteId;
-
   /** Identifies the java group member on the other side of the connection. */
   InternalDistributedMember remoteAddr;
 
@@ -801,7 +796,7 @@ public class Connection implements Runnable {
           }
           if (success) {
             if (this.isReceiver) {
-              needToClose = !owner.getConduit().getMembershipManager().addSurpriseMember(this.remoteAddr, this.remoteId);
+              needToClose = !owner.getConduit().getMembershipManager().addSurpriseMember(this.remoteAddr);
               if (needToClose) {
                 reason = "this member is shunned";
               }
@@ -845,7 +840,7 @@ public class Connection implements Runnable {
    * @param beingSick
    */
   private void asyncClose(boolean beingSick) {
-    // note: remoteId may be null if this is a receiver that hasn't finished its handshake
+    // note: remoteAddr may be null if this is a receiver that hasn't finished its handshake
     
     // we do the close in a background thread because the operation may hang if 
     // there is a problem with the network.  See bug #46659
@@ -1018,8 +1013,7 @@ public class Connection implements Runnable {
   protected static Connection createSender(final MembershipManager mgr,
                                            final ConnectionTable t,
                                            final boolean preserveOrder,
-                                           final Stub key,
-                                           final InternalDistributedMember remoteAddr,
+                                           final DistributedMember remoteAddr,
                                            final boolean sharedResource,
                                            final long startTime,
                                            final long ackTimeout,
@@ -1074,9 +1068,8 @@ public class Connection implements Runnable {
         }
         if (firstTime) {
           firstTime = false;
-          InternalDistributedMember m = mgr.getMemberForStub(key, true);
-          if (m == null) {
-            throw new IOException("Member for stub " + key + " left the group");
+          if (!mgr.memberExists(remoteAddr) || mgr.isShunned(remoteAddr) || mgr.shutdownInProgress()) {
+            throw new IOException("Member " + remoteAddr + " left the system");
           }
         }
         else {
@@ -1084,7 +1077,7 @@ public class Connection implements Runnable {
           // alert listener should not prevent cache operations from continuing
           if (AlertAppender.isThreadAlerting()) {
             // do not change the text of this exception - it is looked for in exception handlers
-            throw new IOException("Cannot form connection to alert listener " + key);
+            throw new IOException("Cannot form connection to alert listener " + remoteAddr);
           }
             
           // Wait briefly...
@@ -1097,20 +1090,19 @@ public class Connection implements Runnable {
             t.getConduit().getCancelCriterion().checkCancelInProgress(ie);
           }
           t.getConduit().getCancelCriterion().checkCancelInProgress(null);
-          InternalDistributedMember m = mgr.getMemberForStub(key, true);
-          if (m == null) {
-            throw new IOException(LocalizedStrings.Connection_MEMBER_FOR_STUB_0_LEFT_THE_GROUP.toLocalizedString(key));
+          if (giveUpOnMember(mgr, remoteAddr)) {
+            throw new IOException(LocalizedStrings.Connection_MEMBER_LEFT_THE_GROUP.toLocalizedString(remoteAddr));
           }
           if (!warningPrinted) {
             warningPrinted = true;
-            logger.warn(LocalizedMessage.create(LocalizedStrings.Connection_CONNECTION_ATTEMPTING_RECONNECT_TO_PEER__0, m));
+            logger.warn(LocalizedMessage.create(LocalizedStrings.Connection_CONNECTION_ATTEMPTING_RECONNECT_TO_PEER__0, remoteAddr));
           }          
           t.getConduit().stats.incReconnectAttempts();
         }
         //create connection
         try {
           conn = null;
-          conn = new Connection(mgr, t, preserveOrder, key, remoteAddr, sharedResource);
+          conn = new Connection(mgr, t, preserveOrder, remoteAddr, sharedResource);
         }
         catch (javax.net.ssl.SSLHandshakeException se) {
           // no need to retry if certificates were rejected
@@ -1118,8 +1110,7 @@ public class Connection implements Runnable {
         }
         catch (IOException ioe) {
           // Only give up if the member leaves the view.
-          InternalDistributedMember m = mgr.getMemberForStub(key, true);
-          if (m == null) {
+          if (giveUpOnMember(mgr, remoteAddr)) {
             throw ioe;
           }
           t.getConduit().getCancelCriterion().checkCancelInProgress(null);
@@ -1130,7 +1121,7 @@ public class Connection implements Runnable {
             connectionErrorLogged = true; // otherwise change to use 100ms intervals causes a lot of these
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.Connection_CONNECTION_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1,
-                new Object[] {sharedResource, preserveOrder, m, ioe}));
+                new Object[] {sharedResource, preserveOrder, remoteAddr, ioe}));
           }
         } // IOException
         finally {
@@ -1146,9 +1137,8 @@ public class Connection implements Runnable {
               // something went wrong while reading the handshake
               // and the socket was closed or this guy sent us a
               // ShutdownMessage
-              InternalDistributedMember m = mgr.getMemberForStub(key, true);
-              if (m == null) {
-                throw new IOException(LocalizedStrings.Connection_MEMBER_FOR_STUB_0_LEFT_THE_GROUP.toLocalizedString(key));
+              if (giveUpOnMember(mgr, remoteAddr)) {
+                throw new IOException(LocalizedStrings.Connection_MEMBER_LEFT_THE_GROUP.toLocalizedString(remoteAddr));
               }
               t.getConduit().getCancelCriterion().checkCancelInProgress(null);
               // no success but no need to log; just retry
@@ -1161,8 +1151,7 @@ public class Connection implements Runnable {
             throw e;
           }
           catch (ConnectionException e) {
-            InternalDistributedMember m = mgr.getMemberForStub(key, true);
-            if (m == null) {
+            if (giveUpOnMember(mgr, remoteAddr)) {
               IOException ioe = new IOException(LocalizedStrings.Connection_HANDSHAKE_FAILED.toLocalizedString());
               ioe.initCause(e);
               throw ioe;
@@ -1170,17 +1159,16 @@ public class Connection implements Runnable {
             t.getConduit().getCancelCriterion().checkCancelInProgress(null);
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.Connection_CONNECTION_HANDSHAKE_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1,
-                new Object[] {sharedResource, preserveOrder, m,e}));
+                new Object[] {sharedResource, preserveOrder, remoteAddr ,e}));
           }
           catch (IOException e) {
-            InternalDistributedMember m = mgr.getMemberForStub(key, true);
-            if (m == null) {
+            if (giveUpOnMember(mgr, remoteAddr)) {
               throw e;
             }
             t.getConduit().getCancelCriterion().checkCancelInProgress(null);
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.Connection_CONNECTION_HANDSHAKE_FAILED_TO_CONNECT_TO_PEER_0_BECAUSE_1,
-                new Object[] {sharedResource, preserveOrder, m,e}));
+                new Object[] {sharedResource, preserveOrder, remoteAddr ,e}));
             if (!sharedResource && "Too many open files".equals(e.getMessage())) {
               t.fileDescriptorsExhausted();
             }
@@ -1220,7 +1208,7 @@ public class Connection implements Runnable {
     if (conn == null) {
       throw new ConnectionException(
         LocalizedStrings.Connection_CONNECTION_FAILED_CONSTRUCTION_FOR_PEER_0
-          .toLocalizedString(mgr.getMemberForStub(key, true)));
+          .toLocalizedString(remoteAddr));
     }
     if (preserveOrder && BATCH_SENDS) {
       conn.createBatchSendBuffer();
@@ -1228,12 +1216,15 @@ public class Connection implements Runnable {
     conn.finishedConnecting = true;
     return conn;
   }
+  
+  private static boolean giveUpOnMember(MembershipManager mgr, DistributedMember remoteAddr) {
+    return !mgr.memberExists(remoteAddr) || mgr.isShunned(remoteAddr) || mgr.shutdownInProgress();
+  }
 
-  private void setRemoteAddr(InternalDistributedMember m, Stub stub) {
+  private void setRemoteAddr(DistributedMember m) {
     this.remoteAddr = this.owner.getDM().getCanonicalId(m);
-    this.remoteId = stub;
     MembershipManager mgr = this.owner.owner.getMembershipManager();
-    mgr.addSurpriseMember(m, stub);
+    mgr.addSurpriseMember(m);
   }
   
   /** creates a new connection to a remote server.
@@ -1243,11 +1234,11 @@ public class Connection implements Runnable {
   private Connection(MembershipManager mgr,
                      ConnectionTable t,
                      boolean preserveOrder,
-                     Stub key,
-                     InternalDistributedMember remoteAddr,
+                     DistributedMember remoteID,
                      boolean sharedResource)
     throws IOException, DistributedSystemDisconnectedException
   {    
+    InternalDistributedMember remoteAddr = (InternalDistributedMember)remoteID;
     if (t == null) {
       throw new IllegalArgumentException(LocalizedStrings.Connection_CONNECTIONTABLE_IS_NULL.toLocalizedString());
     }
@@ -1255,7 +1246,7 @@ public class Connection implements Runnable {
     this.owner = t;
     this.sharedResource = sharedResource;
     this.preserveOrder = preserveOrder;
-    setRemoteAddr(remoteAddr, key);
+    setRemoteAddr(remoteAddr);
     this.conduitIdStr = this.owner.getConduit().getId().toString();
     this.handshakeRead = false;
     this.handshakeCancelled = false;
@@ -1265,7 +1256,7 @@ public class Connection implements Runnable {
 
     // connect to listening socket
 
-    InetSocketAddress addr = new InetSocketAddress(remoteId.getInetAddress(), remoteId.getPort());
+    InetSocketAddress addr = new InetSocketAddress(remoteAddr.getInetAddress(), remoteAddr.getDirectChannelPort());
     if (useNIO()) {
       SocketChannel channel = SocketChannel.open();
       this.owner.addConnectingSocket(channel.socket(), addr.getAddress());
@@ -1325,15 +1316,15 @@ public class Connection implements Runnable {
     else {
       if (TCPConduit.useSSL) {
         // socket = javax.net.ssl.SSLSocketFactory.getDefault()
-        //  .createSocket(remoteId.getInetAddress(), remoteId.getPort());
+        //  .createSocket(remoteAddr.getInetAddress(), remoteAddr.getPort());
         int socketBufferSize = sharedResource ? SMALL_BUFFER_SIZE : this.owner.getConduit().tcpBufferSize;
-        this.socket = SocketCreator.getDefaultInstance().connectForServer( remoteId.getInetAddress(), remoteId.getPort(), socketBufferSize );
+        this.socket = SocketCreator.getDefaultInstance().connectForServer( remoteAddr.getInetAddress(), remoteAddr.getDirectChannelPort(), socketBufferSize );
         // Set the receive buffer size local fields. It has already been set in the socket.
         setSocketBufferSize(this.socket, false, socketBufferSize, true);
         setSendBufferSize(this.socket);
       }
       else {
-        //socket = new Socket(remoteId.getInetAddress(), remoteId.getPort());
+        //socket = new Socket(remoteAddr.getInetAddress(), remoteAddr.getPort());
         Socket s = new Socket();
         this.socket = s;
         s.setTcpNoDelay(true);
@@ -1639,8 +1630,8 @@ public class Connection implements Runnable {
     // we can't wait for the reader thread when running in an IBM JRE.  See
     // bug 41889
     if (this.owner.owner.config.getEnableNetworkPartitionDetection() ||
-        this.owner.owner.getLocalId().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE ||
-        this.owner.owner.getLocalId().getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
+        this.owner.owner.getLocalAddr().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE ||
+        this.owner.owner.getLocalAddr().getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
       isIBM = "IBM Corporation".equals(System.getProperty("java.vm.vendor"));
     }
     {
@@ -1689,16 +1680,16 @@ public class Connection implements Runnable {
               // Only remove endpoint if sender.
               if (this.finishedConnecting) {
                 // only remove endpoint if our constructor finished
-                this.owner.removeEndpoint(this.remoteId, reason);
+                this.owner.removeEndpoint(this.remoteAddr, reason);
               }
             }
           }
           else {
-            this.owner.removeSharedConnection(reason, this.remoteId, this.preserveOrder, this);
+            this.owner.removeSharedConnection(reason, this.remoteAddr, this.preserveOrder, this);
           }
         }
         else if (!this.isReceiver) {
-          this.owner.removeThreadConnection(this.remoteId, this);
+          this.owner.removeThreadConnection(this.remoteAddr, this);
         }
       }
       else {
@@ -1706,10 +1697,10 @@ public class Connection implements Runnable {
         // has never added this Connection to its maps since
         // the calls in this block use our identity to do the removes.
         if (this.sharedResource) {
-          this.owner.removeSharedConnection(reason, this.remoteId, this.preserveOrder, this);
+          this.owner.removeSharedConnection(reason, this.remoteAddr, this.preserveOrder, this);
         }
         else if (!this.isReceiver) {
-          this.owner.removeThreadConnection(this.remoteId, this);
+          this.owner.removeThreadConnection(this.remoteAddr, this);
         }
       }
     }
@@ -1753,7 +1744,7 @@ public class Connection implements Runnable {
     } finally {
       // bug36060: do the socket close within a finally block
       if (logger.isDebugEnabled()) {
-        logger.debug("Stopping {} for {}", p2pReaderName(), remoteId);
+        logger.debug("Stopping {} for {}", p2pReaderName(), remoteAddr);
       }
       initiateSuspicionIfSharedUnordered();
       if (this.isReceiver) {
@@ -2338,8 +2329,7 @@ public class Connection implements Runnable {
                     .toLocalizedString(new Object[]{new Byte(HANDSHAKE_VERSION), new Byte(handShakeByte)}));
               }
               InternalDistributedMember remote = DSFIDFactory.readInternalDistributedMember(dis);
-              Stub stub = new Stub(remote.getInetAddress()/*fix for bug 33615*/, remote.getDirectChannelPort(), remote.getVmViewId());
-              setRemoteAddr(remote, stub);
+              setRemoteAddr(remote);
               Thread.currentThread().setName(LocalizedStrings.Connection_P2P_MESSAGE_READER_FOR_0.toLocalizedString(this.remoteAddr, this.socket.getPort()));
               this.sharedResource = dis.readBoolean();
               this.preserveOrder = dis.readBoolean();
@@ -2377,7 +2367,7 @@ public class Connection implements Runnable {
               }
               
               if (logger.isDebugEnabled()) {
-                logger.debug("{} remoteId is {} {}", p2pReaderName(), this.remoteId,
+                logger.debug("{} remoteAddr is {} {}", p2pReaderName(), this.remoteAddr,
                     (this.remoteVersion != null ? " (" + this.remoteVersion + ')' : ""));
               }
 
@@ -2555,7 +2545,7 @@ public class Connection implements Runnable {
     throws IOException, ConnectionException
   {
     if (!connected) {
-      throw new ConnectionException(LocalizedStrings.Connection_NOT_CONNECTED_TO_0.toLocalizedString(this.remoteId));
+      throw new ConnectionException(LocalizedStrings.Connection_NOT_CONNECTED_TO_0.toLocalizedString(this.remoteAddr));
     }
     if (this.batchFlusher != null) {
       batchSend(buffer);
@@ -2778,7 +2768,7 @@ public class Connection implements Runnable {
         if (this.disconnectRequested) {
           buffer.position(origBufferPos);
           // we have given up so just drop this message.
-          throw new ConnectionException(LocalizedStrings.Connection_FORCED_DISCONNECT_SENT_TO_0.toLocalizedString(this.remoteId));
+          throw new ConnectionException(LocalizedStrings.Connection_FORCED_DISCONNECT_SENT_TO_0.toLocalizedString(this.remoteAddr));
         }
         if (!force && !this.asyncQueuingInProgress) {
           // reset buffer since we will be sending it. This fixes bug 34832
@@ -2980,7 +2970,7 @@ public class Connection implements Runnable {
     }
     DM dm = this.owner.getDM();
     if (dm == null) {
-      this.owner.removeEndpoint(this.remoteId, LocalizedStrings.Connection_NO_DISTRIBUTION_MANAGER.toLocalizedString());
+      this.owner.removeEndpoint(this.remoteAddr, LocalizedStrings.Connection_NO_DISTRIBUTION_MANAGER.toLocalizedString());
       return;
     }
     dm.getMembershipManager().requestMemberRemoval(this.remoteAddr, 
@@ -3001,7 +2991,7 @@ public class Connection implements Runnable {
         return;
       }
     }
-    this.owner.removeEndpoint(this.remoteId, 
+    this.owner.removeEndpoint(this.remoteAddr, 
                               LocalizedStrings.Connection_FORCE_DISCONNECT_TIMED_OUT.toLocalizedString());
     if (dm.getOtherDistributionManagerIds().contains(this.remoteAddr)) {
       if (logger.isDebugEnabled()) {
@@ -3110,7 +3100,7 @@ public class Connection implements Runnable {
       stats.incAsyncThreads(-1);
       stats.incAsyncQueues(-1);
       if (logger.isDebugEnabled()) {
-        logger.debug("runNioPusher terminated id={} from {}/{}", conduitIdStr, remoteId, remoteAddr);
+        logger.debug("runNioPusher terminated id={} from {}/{}", conduitIdStr, remoteAddr, remoteAddr);
       }
     }
     } finally {
@@ -3837,8 +3827,7 @@ public class Connection implements Runnable {
                   throw new IllegalStateException(LocalizedStrings.Connection_DETECTED_WRONG_VERSION_OF_GEMFIRE_PRODUCT_DURING_HANDSHAKE_EXPECTED_0_BUT_FOUND_1.toLocalizedString(new Object[] {new Byte(HANDSHAKE_VERSION), new Byte(handShakeByte)}));
                 }
                 InternalDistributedMember remote = DSFIDFactory.readInternalDistributedMember(dis);
-                Stub stub = new Stub(remote.getInetAddress()/*fix for bug 33615*/, remote.getDirectChannelPort(), remote.getVmViewId());
-                setRemoteAddr(remote, stub);
+                setRemoteAddr(remote);
                 this.sharedResource = dis.readBoolean();
                 this.preserveOrder = dis.readBoolean();
                 this.uniqueId = dis.readLong();
@@ -3897,7 +3886,7 @@ public class Connection implements Runnable {
                 return;
               }
               if (logger.isDebugEnabled()) {
-                logger.debug("P2P handshake remoteId is {}{}", this.remoteId,
+                logger.debug("P2P handshake remoteAddr is {}{}", this.remoteAddr,
                     (this.remoteVersion != null ? " (" + this.remoteVersion + ')' : ""));
               }
               try {
@@ -4031,12 +4020,6 @@ public class Connection implements Runnable {
     this.accessed = true;
   }
 
-  /** returns the ConnectionKey stub representing the other side of
-      this connection (host:port) */
-  public final Stub getRemoteId() {
-    return remoteId;
-  }
-
   /** return the DM id of the guy on the other side of this connection.
    */
   public final InternalDistributedMember getRemoteAddress() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
index bac356c..3816efe 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
@@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.SystemFailure;
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
@@ -60,7 +61,7 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 
 /** <p>ConnectionTable holds all of the Connection objects in a conduit.
     Connections represent a pipe between two endpoints represented
-    by generic Stubs.</p>
+    by generic DistributedMembers.</p>
 
     @author Bruce Schuchardt
     @author Darrel Schneider
@@ -345,7 +346,7 @@ public class ConnectionTable  {
   /**
    * Process a newly created PendingConnection
    * 
-   * @param id Stub on which the connection is created
+   * @param id DistributedMember on which the connection is created
    * @param sharedResource whether the connection is used by multiple threads
    * @param preserveOrder whether to preserve order
    * @param m map to add the connection to
@@ -357,7 +358,7 @@ public class ConnectionTable  {
    * @throws IOException if unable to connect
    * @throws DistributedSystemDisconnectedException
    */
-  private Connection handleNewPendingConnection(Stub id, boolean sharedResource,
+  private Connection handleNewPendingConnection(DistributedMember id, boolean sharedResource,
       boolean preserveOrder,
       Map m, PendingConnection pc, long startTime, long ackThreshold, long ackSAThreshold)
       throws IOException, DistributedSystemDisconnectedException
@@ -366,7 +367,7 @@ public class ConnectionTable  {
     Connection con = null;
     try {
       con = Connection.createSender(owner.getMembershipManager(), this, preserveOrder,
-                                    id, this.owner.getMemberForStub(id, false),
+                                    id,
                                     sharedResource,
                                     startTime, ackThreshold, ackSAThreshold);
       this.owner.stats.incSenders(sharedResource, preserveOrder);
@@ -442,7 +443,7 @@ public class ConnectionTable  {
    * unordered or conserve-sockets
    * note that unordered connections are currently always shared
    * 
-   * @param id the Stub on which we are creating a connection
+   * @param id the DistributedMember on which we are creating a connection
    * @param threadOwnsResources whether unordered conn is owned by the current thread
    * @param preserveOrder whether to preserve order
    * @param startTime the ms clock start time for the operation
@@ -452,7 +453,7 @@ public class ConnectionTable  {
    * @throws IOException if unable to create the connection
    * @throws DistributedSystemDisconnectedException
    */
-  private Connection getUnorderedOrConserveSockets(Stub id, 
+  private Connection getUnorderedOrConserveSockets(DistributedMember id, 
       boolean threadOwnsResources, boolean preserveOrder,
       long startTime, long ackTimeout, long ackSATimeout)
     throws IOException, DistributedSystemDisconnectedException
@@ -527,7 +528,7 @@ public class ConnectionTable  {
    * @throws IOException if the connection could not be created
    * @throws DistributedSystemDisconnectedException
    */
-  Connection getOrderedAndOwned(Stub id, long startTime, long ackTimeout, long ackSATimeout) 
+  Connection getOrderedAndOwned(DistributedMember id, long startTime, long ackTimeout, long ackSATimeout) 
       throws IOException, DistributedSystemDisconnectedException  {
     Connection result = null;
     
@@ -566,7 +567,7 @@ public class ConnectionTable  {
     // OK, we have to create a new connection.
     result = Connection.createSender(owner.getMembershipManager(), 
         this, true /* preserveOrder */, id,
-        this.owner.getMemberForStub(id, false), false /* shared */,
+        false /* shared */,
         startTime, ackTimeout, ackSATimeout);
     if (logger.isDebugEnabled()) {
       logger.debug("ConnectionTable: created an ordered connection: {}", result);
@@ -583,7 +584,7 @@ public class ConnectionTable  {
     
     ArrayList al = (ArrayList)this.threadConnectionMap.get(id);
     if (al == null) {
-      // First connection for this Stub.  Make sure list for this
+      // First connection for this DistributedMember.  Make sure list for this
       // stub is created if it isn't already there.
       al = new ArrayList();
       
@@ -651,7 +652,7 @@ public class ConnectionTable  {
   
   /**
    * Get a new connection
-   * @param id the Stub on which to create the connection
+   * @param id the DistributedMember on which to create the connection
    * @param preserveOrder whether order should be preserved
    * @param startTime the ms clock start time
    * @param ackTimeout the ms ack-wait-threshold, or zero
@@ -660,7 +661,7 @@ public class ConnectionTable  {
    * @throws java.io.IOException if the connection could not be created
    * @throws DistributedSystemDisconnectedException
    */
-  protected Connection get(Stub id, boolean preserveOrder,
+  protected Connection get(DistributedMember id, boolean preserveOrder,
       long startTime, long ackTimeout, long ackSATimeout) 
       throws java.io.IOException, DistributedSystemDisconnectedException
   {
@@ -838,34 +839,38 @@ public class ConnectionTable  {
   /**
    * Return true if our owner already knows that this endpoint is departing 
    */
-  protected boolean isEndpointShuttingDown(Stub stub) {
-    return this.owner.getMemberForStub(stub, true) == null;
+  protected boolean isEndpointShuttingDown(DistributedMember id) {
+    return giveUpOnMember(owner.getDM().getMembershipManager(), id);
   }
   
+  protected boolean giveUpOnMember(MembershipManager mgr, DistributedMember remoteAddr) {
+    return !mgr.memberExists(remoteAddr) || mgr.isShunned(remoteAddr) || mgr.shutdownInProgress();
+  }
+
   /** remove an endpoint and notify the membership manager of the departure */
-  protected void removeEndpoint(Stub stub, String reason) {
+  protected void removeEndpoint(DistributedMember stub, String reason) {
     removeEndpoint(stub, reason, true);
   }
 
-  protected void removeEndpoint(Stub stub, String reason, boolean notifyDisconnect) {
+  protected void removeEndpoint(DistributedMember memberID, String reason, boolean notifyDisconnect) {
     if (this.closed) {
       return;
     }
     boolean needsRemoval = false;
     synchronized (this.orderedConnectionMap) {
-      if (this.orderedConnectionMap.get(stub) != null)
+      if (this.orderedConnectionMap.get(memberID) != null)
         needsRemoval = true;
     }
     if (!needsRemoval) {
       synchronized (this.unorderedConnectionMap) {
-        if (this.unorderedConnectionMap.get(stub) != null)
+        if (this.unorderedConnectionMap.get(memberID) != null)
           needsRemoval = true;
       }
     }
     if (!needsRemoval) {
       ConcurrentMap cm = this.threadConnectionMap;
       if (cm != null) {
-        ArrayList al = (ArrayList)cm.get(stub);
+        ArrayList al = (ArrayList)cm.get(memberID);
         needsRemoval = al != null && al.size() > 0;
       }
     }
@@ -873,14 +878,14 @@ public class ConnectionTable  {
     if (needsRemoval) {
       InternalDistributedMember remoteAddress = null;
       synchronized (this.orderedConnectionMap) {
-        Object c = this.orderedConnectionMap.remove(stub);
+        Object c = this.orderedConnectionMap.remove(memberID);
         if (c instanceof Connection) {
           remoteAddress = ((Connection) c).getRemoteAddress();
         }
         closeCon(reason, c);
       }
       synchronized (this.unorderedConnectionMap) {
-        Object c = this.unorderedConnectionMap.remove(stub);
+        Object c = this.unorderedConnectionMap.remove(memberID);
         if (remoteAddress == null && (c instanceof Connection)) {
           remoteAddress = ((Connection) c).getRemoteAddress();
         }
@@ -890,7 +895,7 @@ public class ConnectionTable  {
       {
         ConcurrentMap cm = this.threadConnectionMap;
         if (cm != null) {
-          ArrayList al = (ArrayList)cm.remove(stub);
+          ArrayList al = (ArrayList)cm.remove(memberID);
           if (al != null) {
             synchronized (al) {
               for (Iterator it=al.iterator(); it.hasNext();) {
@@ -912,7 +917,7 @@ public class ConnectionTable  {
         for (Iterator it=connectingSockets.entrySet().iterator(); it.hasNext(); ) {
           Map.Entry entry = (Map.Entry)it.next();
           ConnectingSocketInfo info = (ConnectingSocketInfo)entry.getValue();
-          if (info.peerAddress.equals(stub.getInetAddress())) {
+          if (info.peerAddress.equals(((InternalDistributedMember)memberID).getInetAddress())) {
             toRemove.add(entry.getKey());
             it.remove();
           }
@@ -925,7 +930,7 @@ public class ConnectionTable  {
         }
         catch (IOException e) {
           if (logger.isDebugEnabled()) {
-            logger.debug("caught exception while trying to close connecting socket for {}", stub, e);
+            logger.debug("caught exception while trying to close connecting socket for {}", memberID, e);
           }
         }
       }
@@ -937,7 +942,7 @@ public class ConnectionTable  {
       synchronized (this.receivers) {
         for (Iterator it=receivers.iterator(); it.hasNext();) {
           Connection con = (Connection)it.next();
-          if (stub.equals(con.getRemoteId())) {
+          if (memberID.equals(con.getRemoteAddress())) {
             it.remove();
             toRemove.add(con);
           }
@@ -947,10 +952,13 @@ public class ConnectionTable  {
         Connection con = (Connection)it.next();
         closeCon(reason, con);
       }
-      // call memberDeparted after doing the closeCon calls
-      // so it can recursively call removeEndpoint
       if (notifyDisconnect) {
-        owner.getMemberForStub(stub, false);
+        // Before the removal of TCPConduit Stub addresses this used
+        // to call MembershipManager.getMemberForStub, which checked
+        // for a shutdown in progress and threw this exception:
+        if (owner.getDM().shutdownInProgress()) {
+          throw new DistributedSystemDisconnectedException("Shutdown in progress", owner.getDM().getMembershipManager().getShutdownCause());
+        }
       }
       
       if (remoteAddress != null) {
@@ -964,11 +972,11 @@ public class ConnectionTable  {
   }
   
   /** check to see if there are still any receiver threads for the given end-point */
-  protected boolean hasReceiversFor(Stub endPoint) {
+  protected boolean hasReceiversFor(DistributedMember endPoint) {
     synchronized (this.receivers) {
       for (Iterator it=receivers.iterator(); it.hasNext();) {
         Connection con = (Connection)it.next();
-        if (endPoint.equals(con.getRemoteId())) {
+        if (endPoint.equals(con.getRemoteAddress())) {
           return true;
         }
       }
@@ -976,7 +984,7 @@ public class ConnectionTable  {
     return false;
   }
   
-  private static void removeFromThreadConMap(ConcurrentMap cm, Stub stub, Connection c) {
+  private static void removeFromThreadConMap(ConcurrentMap cm, DistributedMember stub, Connection c) {
     if (cm != null) {
       ArrayList al = (ArrayList)cm.get(stub);
       if (al != null) {
@@ -986,7 +994,7 @@ public class ConnectionTable  {
       }
     }
   }
-  protected void removeThreadConnection(Stub stub, Connection c) {
+  protected void removeThreadConnection(DistributedMember stub, Connection c) {
     /*if (this.closed) {
       return;
     }*/
@@ -1001,7 +1009,7 @@ public class ConnectionTable  {
       } // synchronized
     } // m != null
   }
-  void removeSharedConnection(String reason, Stub stub, boolean ordered, Connection c) {
+  void removeSharedConnection(String reason, DistributedMember stub, boolean ordered, Connection c) {
     if (this.closed) {
       return;
     }
@@ -1054,7 +1062,7 @@ public class ConnectionTable  {
        Iterator it = m.entrySet().iterator();
        while (it.hasNext()) {
          Map.Entry me = (Map.Entry)it.next();
-         Stub stub = (Stub)me.getKey();
+         DistributedMember stub = (DistributedMember)me.getKey();
          Connection c = (Connection)me.getValue();
          removeFromThreadConMap(this.threadConnectionMap, stub, c);
          it.remove();
@@ -1079,7 +1087,7 @@ public class ConnectionTable  {
    * from being formed or new messages from being sent
    * @since 5.1
    */
-  protected void getThreadOwnedOrderedConnectionState(Stub member,
+  protected void getThreadOwnedOrderedConnectionState(DistributedMember member,
       Map result) {
 
     ConcurrentMap cm = this.threadConnectionMap;
@@ -1105,7 +1113,7 @@ public class ConnectionTable  {
    * wait for the given incoming connections to receive at least the associated
    * number of messages
    */
-  protected void waitForThreadOwnedOrderedConnectionState(Stub member,
+  protected void waitForThreadOwnedOrderedConnectionState(DistributedMember member,
       Map connectionStates) throws InterruptedException {
     if (Thread.interrupted()) throw new InterruptedException(); // wisest to do this before the synchronize below
     List r = null;
@@ -1115,14 +1123,14 @@ public class ConnectionTable  {
     for (Iterator it=r.iterator(); it.hasNext();) {
       Connection con = (Connection)it.next();
       if (!con.stopped && !con.isClosing() && !con.getOriginatedHere() && con.getPreserveOrder()
-          && member.equals(con.getRemoteId())) {
+          && member.equals(con.getRemoteAddress())) {
         Long state = (Long)connectionStates.remove(Long.valueOf(con.getUniqueId()));
         if (state != null) {
           long count = state.longValue();
           while (!con.stopped && !con.isClosing() && con.getMessagesReceived() < count) {
             if (logger.isDebugEnabled()) {
               logger.debug("Waiting for connection {}/{} currently={} need={}", 
-                  con.getRemoteId(), con.getUniqueId(), con.getMessagesReceived(), count);
+                  con.getRemoteAddress(), con.getUniqueId(), con.getMessagesReceived(), count);
             }
             Thread.sleep(100);
           }
@@ -1230,11 +1238,11 @@ public class ConnectionTable  {
     /**
      * the stub we are connecting to
      */
-    private final Stub id;
+    private final DistributedMember id;
     
     private final Thread connectingThread;
     
-    public PendingConnection(boolean preserveOrder, Stub id) {
+    public PendingConnection(boolean preserveOrder, DistributedMember id) {
       this.preserveOrder = preserveOrder;
       this.id = id;
       this.connectingThread = Thread.currentThread();
@@ -1279,10 +1287,9 @@ public class ConnectionTable  {
 
       boolean severeAlertIssued = false;
       boolean suspected = false;
-      InternalDistributedMember targetMember = null;
+      DistributedMember targetMember = null;
       if (ackSATimeout > 0) {
-        targetMember =
-          ((GMSMembershipManager)mgr).getMemberForStub(this.id, false);
+        targetMember = this.id;
       }
 
       for (;;) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
index 5cd426f..a954814 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/MemberShunnedException.java
@@ -18,6 +18,7 @@
 package com.gemstone.gemfire.internal.tcp;
 
 import com.gemstone.gemfire.GemFireException;
+import com.gemstone.gemfire.distributed.DistributedMember;
 
 /**
  * MemberShunnedException may be thrown to prevent ack-ing a message
@@ -28,13 +29,13 @@ import com.gemstone.gemfire.GemFireException;
 public class MemberShunnedException extends GemFireException
 {
   private static final long serialVersionUID = -8453126202477831557L;
-  private Stub member;
+  private DistributedMember member;
   
   /**
    * constructor
    * @param member the member that was shunned
    */
-  public MemberShunnedException(Stub member) {
+  public MemberShunnedException(DistributedMember member) {
     super("");
     this.member = member;
   }
@@ -42,7 +43,7 @@ public class MemberShunnedException extends GemFireException
   /**
    * @return the member that was shunned
    */
-  public Stub getShunnedMember() {
+  public DistributedMember getShunnedMember() {
     return this.member;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
index fd495d9..cd711e7 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ServerDelegate.java
@@ -16,6 +16,7 @@
  */
 package com.gemstone.gemfire.internal.tcp;
 
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.membership.*;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
@@ -34,7 +35,7 @@ import com.gemstone.gemfire.i18n.LogWriterI18n;
 public interface ServerDelegate {
 
   public void receive( DistributionMessage message, int bytesRead,
-                       Stub connId );
+                       DistributedMember connId );
 
   public LogWriterI18n getLogger();
 
@@ -42,5 +43,5 @@ public interface ServerDelegate {
    * Called when a possibly new member is detected by receiving a direct channel
    * message from him.
    */
-  public void newMemberConnected(InternalDistributedMember member, Stub id);
+  public void newMemberConnected(InternalDistributedMember member);
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5b35e43f/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java
deleted file mode 100644
index 2e4b91b..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/Stub.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.tcp;
-
-import java.io.*;
-import java.net.*;
-
-import com.gemstone.gemfire.DataSerializable;
-import com.gemstone.gemfire.DataSerializer;
-import com.gemstone.gemfire.internal.InternalDataSerializer;
-
-/** Stub represents an ip address and port.
-
-    @author Bruce Schuchardt
-    @since 2.0
-   
- */
-
-public class Stub implements Externalizable, DataSerializable
-{
-  private InetAddress inAddr;
-  private int port;
-  private int viewID;
-
-  public Stub() {
-    // public default needed for deserialization
-  }
-  
-  public Stub(InetAddress addr, int port, int vmViewID) {
-    viewID = vmViewID;
-    inAddr = addr;
-    this.port = port;
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (o == this) {
-      return true;
-    }
-    if (o instanceof Stub) {
-      Stub s = (Stub)o;
-      boolean result;
-      if (inAddr == null)
-        result = s.inAddr == null;
-      else
-        result = inAddr.equals(s.inAddr);
-      result = result && port == s.port;
-      if (this.viewID != 0 && s.viewID != 0) {
-        result = result && (this.viewID == s.viewID);
-      }
-      return result;
-    }
-    else {
-      return false;
-    }
-  }
-  
-  // hashCode equates to the address hashCode for fast connection lookup
-  @Override
-  public int hashCode() {
-    // do not use viewID in hashCode because it is changed after creating a stub
-    int result = 0;
-    // result += inAddr.hashCode(); // useless
-    result += port;
-    return result;
-  }
-  
-  public void setViewID(int viewID) {
-    this.viewID = viewID;
-  }
-  
-  public int getPort() {
-    return port;
-  }
-  
-  public int getViewID() {
-    return this.viewID;
-  }
-  
-  public InetAddress getInetAddress() {
-    return inAddr;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder(80);
-    sb.append("tcp://");
-    if (inAddr == null)
-      sb.append("<null>");
-    else
-      sb.append(inAddr.toString());
-    if (this.viewID != 0) {
-      sb.append("<v"+this.viewID+">");
-    }
-    sb.append(":" + port);
-    return sb.toString();
-  }
-  
-  /**
-   * Writes the contents of this <code>Stub</code> to a
-   * <code>DataOutput</code>. 
-   *
-   * @since 3.0
-   */
-  public void toData(DataOutput out) 
-    throws IOException
-  {
-    DataSerializer.writeInetAddress(inAddr, out);
-    out.writeInt(port);
-    out.writeInt(viewID);
-  }
-  
-  /**
-   * Reads the contents of this <code>Stub</code> from a
-   * <code>DataOutput</code>. 
-   *
-   * @since 3.0
-   */
-  public void fromData(DataInput in)
-    throws IOException, ClassNotFoundException
-  {
-    inAddr = DataSerializer.readInetAddress(in);
-    this.port = in.readInt();
-    this.viewID = in.readInt();
-  }
-
-  /**
-   * static factory method
-   * @since 5.0.2
-   */
-  public static Stub createFromData(DataInput in)
-    throws IOException, ClassNotFoundException
-  {
-    Stub result = new Stub();
-    InternalDataSerializer.invokeFromData(result, in);
-    return result;
-  }
-  
-  public void writeExternal(ObjectOutput os) 
-    throws IOException
-  {
-    this.toData(os);
-  }
-  
-  public void readExternal(ObjectInput is)
-    throws IOException, ClassNotFoundException
-  {
-    this.fromData(is);
-  }
-}


[20/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/font/fontawesome-webfont-svg.svg
----------------------------------------------------------------------
diff --git a/gemfire-site/content/font/fontawesome-webfont-svg.svg b/gemfire-site/content/font/fontawesome-webfont-svg.svg
deleted file mode 100755
index 2edb4ec..0000000
--- a/gemfire-site/content/font/fontawesome-webfont-svg.svg
+++ /dev/null
@@ -1,399 +0,0 @@
-<?xml version="1.0" standalone="no"?>
-<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
-<svg xmlns="http://www.w3.org/2000/svg">
-<metadata></metadata>
-<defs>
-<font id="fontawesomeregular" horiz-adv-x="1536" >
-<font-face units-per-em="1792" ascent="1536" descent="-256" />
-<missing-glyph horiz-adv-x="448" />
-<glyph unicode=" "  horiz-adv-x="448" />
-<glyph unicode="&#x09;" horiz-adv-x="448" />
-<glyph unicode="&#xa0;" horiz-adv-x="448" />
-<glyph unicode="&#xa8;" horiz-adv-x="1792" />
-<glyph unicode="&#xa9;" horiz-adv-x="1792" />
-<glyph unicode="&#xae;" horiz-adv-x="1792" />
-<glyph unicode="&#xb4;" horiz-adv-x="1792" />
-<glyph unicode="&#xc6;" horiz-adv-x="1792" />
-<glyph unicode="&#x2000;" horiz-adv-x="768" />
-<glyph unicode="&#x2001;" />
-<glyph unicode="&#x2002;" horiz-adv-x="768" />
-<glyph unicode="&#x2003;" />
-<glyph unicode="&#x2004;" horiz-adv-x="512" />
-<glyph unicode="&#x2005;" horiz-adv-x="384" />
-<glyph unicode="&#x2006;" horiz-adv-x="256" />
-<glyph unicode="&#x2007;" horiz-adv-x="256" />
-<glyph unicode="&#x2008;" horiz-adv-x="192" />
-<glyph unicode="&#x2009;" horiz-adv-x="307" />
-<glyph unicode="&#x200a;" horiz-adv-x="85" />
-<glyph unicode="&#x202f;" horiz-adv-x="307" />
-<glyph unicode="&#x205f;" horiz-adv-x="384" />
-<glyph unicode="&#x2122;" horiz-adv-x="1792" />
-<glyph unicode="&#x221e;" horiz-adv-x="1792" />
-<glyph unicode="&#x2260;" horiz-adv-x="1792" />
-<glyph unicode="&#xe000;" horiz-adv-x="500" d="M0 0z" />
-<glyph unicode="&#xf000;" horiz-adv-x="1792" d="M1699 1350q0 -35 -43 -78l-632 -632v-768h320q26 0 45 -19t19 -45t-19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45t45 19h320v768l-632 632q-43 43 -43 78q0 23 18 36.5t38 17.5t43 4h1408q23 0 43 -4t38 -17.5t18 -36.5z" />
-<glyph unicode="&#xf001;" d="M1536 1312v-1120q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v537l-768 -237v-709q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89 t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v967q0 31 19 56.5t49 35.5l832 256q12 4 28 4q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf002;" horiz-adv-x="1664" d="M1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -52 -38 -90t-90 -38q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5 t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
-<glyph unicode="&#xf003;" horiz-adv-x="1792" d="M1664 32v768q-32 -36 -69 -66q-268 -206 -426 -338q-51 -43 -83 -67t-86.5 -48.5t-102.5 -24.5h-1h-1q-48 0 -102.5 24.5t-86.5 48.5t-83 67q-158 132 -426 338q-37 30 -69 66v-768q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1664 1083v11v13.5t-0.5 13 t-3 12.5t-5.5 9t-9 7.5t-14 2.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5q0 -168 147 -284q193 -152 401 -317q6 -5 35 -29.5t46 -37.5t44.5 -31.5t50.5 -27.5t43 -9h1h1q20 0 43 9t50.5 27.5t44.5 31.5t46 37.5t35 29.5q208 165 401 317q54 43 100.5 115.5t46.5 131.5z M1792 1120v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf004;" horiz-adv-x="1792" d="M896 -128q-26 0 -44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5q224 0 351 -124t127 -344q0 -221 -229 -450l-623 -600 q-18 -18 -44 -18z" />
-<glyph unicode="&#xf005;" horiz-adv-x="1664" d="M1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -21 -10.5 -35.5t-30.5 -14.5q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455 l502 -73q56 -9 56 -46z" />
-<glyph unicode="&#xf006;" horiz-adv-x="1664" d="M1137 532l306 297l-422 62l-189 382l-189 -382l-422 -62l306 -297l-73 -421l378 199l377 -199zM1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -50 -41 -50q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500 l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455l502 -73q56 -9 56 -46z" />
-<glyph unicode="&#xf007;" horiz-adv-x="1408" d="M1408 131q0 -120 -73 -189.5t-194 -69.5h-874q-121 0 -194 69.5t-73 189.5q0 53 3.5 103.5t14 109t26.5 108.5t43 97.5t62 81t85.5 53.5t111.5 20q9 0 42 -21.5t74.5 -48t108 -48t133.5 -21.5t133.5 21.5t108 48t74.5 48t42 21.5q61 0 111.5 -20t85.5 -53.5t62 -81 t43 -97.5t26.5 -108.5t14 -109t3.5 -103.5zM1088 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5z" />
-<glyph unicode="&#xf008;" horiz-adv-x="1920" d="M384 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 320v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 704v128q0 26 -19 45t-45 19h-128 q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 -64v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM384 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45 t45 -19h128q26 0 45 19t19 45zM1792 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 704v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1792 320v128 q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1792 704v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t1
 9 45zM1792 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19 t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1920 1248v-1344q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1344q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf009;" horiz-adv-x="1664" d="M768 512v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM768 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 512v-384q0 -52 -38 -90t-90 -38 h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" />
-<glyph unicode="&#xf00a;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 288v-192q0 -40 -28 -68t-68 -28h-320 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192 q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28
 t28 -68z" />
-<glyph unicode="&#xf00b;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-960 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h960q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf00c;" horiz-adv-x="1792" d="M1671 970q0 -40 -28 -68l-724 -724l-136 -136q-28 -28 -68 -28t-68 28l-136 136l-362 362q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -295l656 657q28 28 68 28t68 -28l136 -136q28 -28 28 -68z" />
-<glyph unicode="&#xf00d;" horiz-adv-x="1408" d="M1298 214q0 -40 -28 -68l-136 -136q-28 -28 -68 -28t-68 28l-294 294l-294 -294q-28 -28 -68 -28t-68 28l-136 136q-28 28 -28 68t28 68l294 294l-294 294q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -294l294 294q28 28 68 28t68 -28l136 -136q28 -28 28 -68 t-28 -68l-294 -294l294 -294q28 -28 28 -68z" />
-<glyph unicode="&#xf00e;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-224q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v224h-224q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h224v224q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5v-224h224 q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5 t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
-<glyph unicode="&#xf010;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-576q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h576q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5z M1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z " />
-<glyph unicode="&#xf011;" d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61t-298 61t-245 164t-164 245t-61 298q0 182 80.5 343t226.5 270q43 32 95.5 25t83.5 -50q32 -42 24.5 -94.5t-49.5 -84.5q-98 -74 -151.5 -181t-53.5 -228q0 -104 40.5 -198.5t109.5 -163.5t163.5 -109.5 t198.5 -40.5t198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5q0 121 -53.5 228t-151.5 181q-42 32 -49.5 84.5t24.5 94.5q31 43 84 50t95 -25q146 -109 226.5 -270t80.5 -343zM896 1408v-640q0 -52 -38 -90t-90 -38t-90 38t-38 90v640q0 52 38 90t90 38t90 -38t38 -90z" />
-<glyph unicode="&#xf012;" horiz-adv-x="1792" d="M256 96v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM640 224v-320q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1024 480v-576q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23 v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1408 864v-960q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 1376v-1472q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v1472q0 14 9 23t23 9h192q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf013;" d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1536 749v-222q0 -12 -8 -23t-20 -13l-185 -28q-19 -54 -39 -91q35 -50 107 -138q10 -12 10 -25t-9 -23q-27 -37 -99 -108t-94 -71q-12 0 -26 9l-138 108q-44 -23 -91 -38 q-16 -136 -29 -186q-7 -28 -36 -28h-222q-14 0 -24.5 8.5t-11.5 21.5l-28 184q-49 16 -90 37l-141 -107q-10 -9 -25 -9q-14 0 -25 11q-126 114 -165 168q-7 10 -7 23q0 12 8 23q15 21 51 66.5t54 70.5q-27 50 -41 99l-183 27q-13 2 -21 12.5t-8 23.5v222q0 12 8 23t19 13 l186 28q14 46 39 92q-40 57 -107 138q-10 12 -10 24q0 10 9 23q26 36 98.5 107.5t94.5 71.5q13 0 26 -10l138 -107q44 23 91 38q16 136 29 186q7 28 36 28h222q14 0 24.5 -8.5t11.5 -21.5l28 -184q49 -16 90 -37l142 107q9 9 24 9q13 0 25 -10q129 -119 165 -170q7 -8 7 -22 q0 -12 -8 -23q-15 -21 -51 -66.5t-54 -70.5q26 -50 41 -98l183 -28q13 -2 21 -12.5t8 -23.5z" />
-<glyph unicode="&#xf014;" horiz-adv-x="1408" d="M512 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM768 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1024 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1152 76v948h-896v-948q0 -22 7 -40.5t14.5 -27t10.5 -8.5h832q3 0 10.5 8.5t14.5 27t7 40.5zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q0 -14 -9 -23t-23 -9h-96v-948q0 -83 -47 -143.5t-113 -60.5h-832 q-66 0 -113 58.5t-47 141.5v952h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h309l70 167q15 37 54 63t79 26h320q40 0 79 -26t54 -63l70 -167h309q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf015;" horiz-adv-x="1664" d="M1408 544v-480q0 -26 -19 -45t-45 -19h-384v384h-256v-384h-384q-26 0 -45 19t-19 45v480q0 1 0.5 3t0.5 3l575 474l575 -474q1 -2 1 -6zM1631 613l-62 -74q-8 -9 -21 -11h-3q-13 0 -21 7l-692 577l-692 -577q-12 -8 -24 -7q-13 2 -21 11l-62 74q-8 10 -7 23.5t11 21.5 l719 599q32 26 76 26t76 -26l244 -204v195q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-408l219 -182q10 -8 11 -21.5t-7 -23.5z" />
-<glyph unicode="&#xf016;" horiz-adv-x="1280" d="M128 0h1024v768h-416q-40 0 -68 28t-28 68v416h-512v-1280zM768 896h376q-10 29 -22 41l-313 313q-12 12 -41 22v-376zM1280 864v-896q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h640q40 0 88 -20t76 -48l312 -312q28 -28 48 -76t20 -88z " />
-<glyph unicode="&#xf017;" d="M896 992v-448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf018;" horiz-adv-x="1920" d="M1111 540v4l-24 320q-1 13 -11 22.5t-23 9.5h-186q-13 0 -23 -9.5t-11 -22.5l-24 -320v-4q-1 -12 8 -20t21 -8h244q12 0 21 8t8 20zM1870 73q0 -73 -46 -73h-704q13 0 22 9.5t8 22.5l-20 256q-1 13 -11 22.5t-23 9.5h-272q-13 0 -23 -9.5t-11 -22.5l-20 -256 q-1 -13 8 -22.5t22 -9.5h-704q-46 0 -46 73q0 54 26 116l417 1044q8 19 26 33t38 14h339q-13 0 -23 -9.5t-11 -22.5l-15 -192q-1 -14 8 -23t22 -9h166q13 0 22 9t8 23l-15 192q-1 13 -11 22.5t-23 9.5h339q20 0 38 -14t26 -33l417 -1044q26 -62 26 -116z" />
-<glyph unicode="&#xf019;" horiz-adv-x="1664" d="M1280 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 416v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h465l135 -136 q58 -56 136 -56t136 56l136 136h464q40 0 68 -28t28 -68zM1339 985q17 -41 -14 -70l-448 -448q-18 -19 -45 -19t-45 19l-448 448q-31 29 -14 70q17 39 59 39h256v448q0 26 19 45t45 19h256q26 0 45 -19t19 -45v-448h256q42 0 59 -39z" />
-<glyph unicode="&#xf01a;" d="M1120 608q0 -12 -10 -24l-319 -319q-11 -9 -23 -9t-23 9l-320 320q-15 16 -7 35q8 20 30 20h192v352q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-352h192q14 0 23 -9t9 -23zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273 t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01b;" d="M1118 660q-8 -20 -30 -20h-192v-352q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v352h-192q-14 0 -23 9t-9 23q0 12 10 24l319 319q11 9 23 9t23 -9l320 -320q15 -16 7 -35zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198 t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01c;" d="M1023 576h316q-1 3 -2.5 8t-2.5 8l-212 496h-708l-212 -496q-1 -2 -2.5 -8t-2.5 -8h316l95 -192h320zM1536 546v-482q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v482q0 62 25 123l238 552q10 25 36.5 42t52.5 17h832q26 0 52.5 -17t36.5 -42l238 -552 q25 -61 25 -123z" />
-<glyph unicode="&#xf01d;" d="M1184 640q0 -37 -32 -55l-544 -320q-15 -9 -32 -9q-16 0 -32 8q-32 19 -32 56v640q0 37 32 56q33 18 64 -1l544 -320q32 -18 32 -55zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01e;" d="M1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l138 138q-148 137 -349 137q-104 0 -198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5q119 0 225 52t179 147q7 10 23 12q14 0 25 -9 l137 -138q9 -8 9.5 -20.5t-7.5 -22.5q-109 -132 -264 -204.5t-327 -72.5q-156 0 -298 61t-245 164t-164 245t-61 298t61 298t164 245t245 164t298 61q147 0 284.5 -55.5t244.5 -156.5l130 129q29 31 70 14q39 -17 39 -59z" />
-<glyph unicode="&#xf021;" d="M1511 480q0 -5 -1 -7q-64 -268 -268 -434.5t-478 -166.5q-146 0 -282.5 55t-243.5 157l-129 -129q-19 -19 -45 -19t-45 19t-19 45v448q0 26 19 45t45 19h448q26 0 45 -19t19 -45t-19 -45l-137 -137q71 -66 161 -102t187 -36q134 0 250 65t186 179q11 17 53 117 q8 23 30 23h192q13 0 22.5 -9.5t9.5 -22.5zM1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-26 0 -45 19t-19 45t19 45l138 138q-148 137 -349 137q-134 0 -250 -65t-186 -179q-11 -17 -53 -117q-8 -23 -30 -23h-199q-13 0 -22.5 9.5t-9.5 22.5v7q65 268 270 434.5t480 166.5 q146 0 284 -55.5t245 -156.5l130 129q19 19 45 19t45 -19t19 -45z" />
-<glyph unicode="&#xf022;" horiz-adv-x="1792" d="M384 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M384 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1536 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5z M1536 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5zM1536 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5 t9.5 -22.5zM1664 160v832q0 13 -9.5 22.5t-22.5 9.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5v-832q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1792 1248v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -1
 13 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47 t47 -113z" />
-<glyph unicode="&#xf023;" horiz-adv-x="1152" d="M320 768h512v192q0 106 -75 181t-181 75t-181 -75t-75 -181v-192zM1152 672v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v192q0 184 132 316t316 132t316 -132t132 -316v-192h32q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf024;" horiz-adv-x="1792" d="M320 1280q0 -72 -64 -110v-1266q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v1266q-64 38 -64 110q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -25 -12.5 -38.5t-39.5 -27.5q-215 -116 -369 -116q-61 0 -123.5 22t-108.5 48 t-115.5 48t-142.5 22q-192 0 -464 -146q-17 -9 -33 -9q-26 0 -45 19t-19 45v742q0 32 31 55q21 14 79 43q236 120 421 120q107 0 200 -29t219 -88q38 -19 88 -19q54 0 117.5 21t110 47t88 47t54.5 21q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf025;" horiz-adv-x="1664" d="M1664 650q0 -166 -60 -314l-20 -49l-185 -33q-22 -83 -90.5 -136.5t-156.5 -53.5v-32q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-32q71 0 130 -35.5t93 -95.5l68 12q29 95 29 193q0 148 -88 279t-236.5 209t-315.5 78 t-315.5 -78t-236.5 -209t-88 -279q0 -98 29 -193l68 -12q34 60 93 95.5t130 35.5v32q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v32q-88 0 -156.5 53.5t-90.5 136.5l-185 33l-20 49q-60 148 -60 314q0 151 67 291t179 242.5 t266 163.5t320 61t320 -61t266 -163.5t179 -242.5t67 -291z" />
-<glyph unicode="&#xf026;" horiz-adv-x="768" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45z" />
-<glyph unicode="&#xf027;" horiz-adv-x="1152" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142z" />
-<glyph unicode="&#xf028;" horiz-adv-x="1664" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142zM1408 640q0 -153 -85 -282.5t-225 -188.5q-13 -5 -25 -5q-27 0 -46 19t-19 45q0 39 39 59q56 29 76 44q74 54 115.5 135.5t41.5 173.5t-41.5 173.5 t-115.5 135.5q-20 15 -76 44q-39 20 -39 59q0 26 19 45t45 19q13 0 26 -5q140 -59 225 -188.5t85 -282.5zM1664 640q0 -230 -127 -422.5t-338 -283.5q-13 -5 -26 -5q-26 0 -45 19t-19 45q0 36 39 59q7 4 22.5 10.5t22.5 10.5q46 25 82 51q123 91 192 227t69 289t-69 289 t-192 227q-36 26 -82 51q-7 4 -22.5 10.5t-22.5 10.5q-39 23 -39 59q0 26 19 45t45 19q13 0 26 -5q211 -91 338 -283.5t127 -422.5z" />
-<glyph unicode="&#xf029;" horiz-adv-x="1408" d="M384 384v-128h-128v128h128zM384 1152v-128h-128v128h128zM1152 1152v-128h-128v128h128zM128 129h384v383h-384v-383zM128 896h384v384h-384v-384zM896 896h384v384h-384v-384zM640 640v-640h-640v640h640zM1152 128v-128h-128v128h128zM1408 128v-128h-128v128h128z M1408 640v-384h-384v128h-128v-384h-128v640h384v-128h128v128h128zM640 1408v-640h-640v640h640zM1408 1408v-640h-640v640h640z" />
-<glyph unicode="&#xf02a;" horiz-adv-x="1792" d="M63 0h-63v1408h63v-1408zM126 1h-32v1407h32v-1407zM220 1h-31v1407h31v-1407zM377 1h-31v1407h31v-1407zM534 1h-62v1407h62v-1407zM660 1h-31v1407h31v-1407zM723 1h-31v1407h31v-1407zM786 1h-31v1407h31v-1407zM943 1h-63v1407h63v-1407zM1100 1h-63v1407h63v-1407z M1226 1h-63v1407h63v-1407zM1352 1h-63v1407h63v-1407zM1446 1h-63v1407h63v-1407zM1635 1h-94v1407h94v-1407zM1698 1h-32v1407h32v-1407zM1792 0h-63v1408h63v-1408z" />
-<glyph unicode="&#xf02b;" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91z" />
-<glyph unicode="&#xf02c;" horiz-adv-x="1920" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91zM1899 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-36 0 -59 14t-53 45l470 470q37 37 37 90q0 52 -37 91l-715 714q-38 38 -102 64.5t-117 26.5h224q53 0 117 -26.5t102 -64.5l715 -714q37 -39 37 -91z" />
-<glyph unicode="&#xf02d;" horiz-adv-x="1664" d="M1639 1058q40 -57 18 -129l-275 -906q-19 -64 -76.5 -107.5t-122.5 -43.5h-923q-77 0 -148.5 53.5t-99.5 131.5q-24 67 -2 127q0 4 3 27t4 37q1 8 -3 21.5t-3 19.5q2 11 8 21t16.5 23.5t16.5 23.5q23 38 45 91.5t30 91.5q3 10 0.5 30t-0.5 28q3 11 17 28t17 23 q21 36 42 92t25 90q1 9 -2.5 32t0.5 28q4 13 22 30.5t22 22.5q19 26 42.5 84.5t27.5 96.5q1 8 -3 25.5t-2 26.5q2 8 9 18t18 23t17 21q8 12 16.5 30.5t15 35t16 36t19.5 32t26.5 23.5t36 11.5t47.5 -5.5l-1 -3q38 9 51 9h761q74 0 114 -56t18 -130l-274 -906 q-36 -119 -71.5 -153.5t-128.5 -34.5h-869q-27 0 -38 -15q-11 -16 -1 -43q24 -70 144 -70h923q29 0 56 15.5t35 41.5l300 987q7 22 5 57q38 -15 59 -43zM575 1056q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5 t-16.5 -22.5zM492 800q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5t-16.5 -22.5z" />
-<glyph unicode="&#xf02e;" horiz-adv-x="1280" d="M1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
-<glyph unicode="&#xf02f;" horiz-adv-x="1664" d="M384 0h896v256h-896v-256zM384 640h896v384h-160q-40 0 -68 28t-28 68v160h-640v-640zM1536 576q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 576v-416q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-160q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68 v160h-224q-13 0 -22.5 9.5t-9.5 22.5v416q0 79 56.5 135.5t135.5 56.5h64v544q0 40 28 68t68 28h672q40 0 88 -20t76 -48l152 -152q28 -28 48 -76t20 -88v-256h64q79 0 135.5 -56.5t56.5 -135.5z" />
-<glyph unicode="&#xf030;" horiz-adv-x="1920" d="M960 864q119 0 203.5 -84.5t84.5 -203.5t-84.5 -203.5t-203.5 -84.5t-203.5 84.5t-84.5 203.5t84.5 203.5t203.5 84.5zM1664 1280q106 0 181 -75t75 -181v-896q0 -106 -75 -181t-181 -75h-1408q-106 0 -181 75t-75 181v896q0 106 75 181t181 75h224l51 136 q19 49 69.5 84.5t103.5 35.5h512q53 0 103.5 -35.5t69.5 -84.5l51 -136h224zM960 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" />
-<glyph unicode="&#xf031;" horiz-adv-x="1664" d="M725 977l-170 -450q73 -1 153.5 -2t119 -1.5t52.5 -0.5l29 2q-32 95 -92 241q-53 132 -92 211zM21 -128h-21l2 79q22 7 80 18q89 16 110 31q20 16 48 68l237 616l280 724h75h53l11 -21l205 -480q103 -242 124 -297q39 -102 96 -235q26 -58 65 -164q24 -67 65 -149 q22 -49 35 -57q22 -19 69 -23q47 -6 103 -27q6 -39 6 -57q0 -14 -1 -26q-80 0 -192 8q-93 8 -189 8q-79 0 -135 -2l-200 -11l-58 -2q0 45 4 78l131 28q56 13 68 23q12 12 12 27t-6 32l-47 114l-92 228l-450 2q-29 -65 -104 -274q-23 -64 -23 -84q0 -31 17 -43 q26 -21 103 -32q3 0 13.5 -2t30 -5t40.5 -6q1 -28 1 -58q0 -17 -2 -27q-66 0 -349 20l-48 -8q-81 -14 -167 -14z" />
-<glyph unicode="&#xf032;" horiz-adv-x="1408" d="M555 15q76 -32 140 -32q131 0 216 41t122 113q38 70 38 181q0 114 -41 180q-58 94 -141 126q-80 32 -247 32q-74 0 -101 -10v-144l-1 -173l3 -270q0 -15 12 -44zM541 761q43 -7 109 -7q175 0 264 65t89 224q0 112 -85 187q-84 75 -255 75q-52 0 -130 -13q0 -44 2 -77 q7 -122 6 -279l-1 -98q0 -43 1 -77zM0 -128l2 94q45 9 68 12q77 12 123 31q17 27 21 51q9 66 9 194l-2 497q-5 256 -9 404q-1 87 -11 109q-1 4 -12 12q-18 12 -69 15q-30 2 -114 13l-4 83l260 6l380 13l45 1q5 0 14 0.5t14 0.5q1 0 21.5 -0.5t40.5 -0.5h74q88 0 191 -27 q43 -13 96 -39q57 -29 102 -76q44 -47 65 -104t21 -122q0 -70 -32 -128t-95 -105q-26 -20 -150 -77q177 -41 267 -146q92 -106 92 -236q0 -76 -29 -161q-21 -62 -71 -117q-66 -72 -140 -108q-73 -36 -203 -60q-82 -15 -198 -11l-197 4q-84 2 -298 -11q-33 -3 -272 -11z" />
-<glyph unicode="&#xf033;" horiz-adv-x="1024" d="M0 -126l17 85q4 1 77 20q76 19 116 39q29 37 41 101l27 139l56 268l12 64q8 44 17 84.5t16 67t12.5 46.5t9 30.5t3.5 11.5l29 157l16 63l22 135l8 50v38q-41 22 -144 28q-28 2 -38 4l19 103l317 -14q39 -2 73 -2q66 0 214 9q33 2 68 4.5t36 2.5q-2 -19 -6 -38 q-7 -29 -13 -51q-55 -19 -109 -31q-64 -16 -101 -31q-12 -31 -24 -88q-9 -44 -13 -82q-44 -199 -66 -306l-61 -311l-38 -158l-43 -235l-12 -45q-2 -7 1 -27q64 -15 119 -21q36 -5 66 -10q-1 -29 -7 -58q-7 -31 -9 -41q-18 0 -23 -1q-24 -2 -42 -2q-9 0 -28 3q-19 4 -145 17 l-198 2q-41 1 -174 -11q-74 -7 -98 -9z" />
-<glyph unicode="&#xf034;" horiz-adv-x="1792" d="M81 1407l54 -27q20 -5 211 -5h130l19 3l115 1l215 -1h293l34 -2q14 -1 28 7t21 16l7 8l42 1q15 0 28 -1v-104.5t1 -131.5l1 -100l-1 -58q0 -32 -4 -51q-39 -15 -68 -18q-25 43 -54 128q-8 24 -15.5 62.5t-11.5 65.5t-6 29q-13 15 -27 19q-7 2 -42.5 2t-103.5 -1t-111 -1 q-34 0 -67 -5q-10 -97 -8 -136l1 -152v-332l3 -359l-1 -147q-1 -46 11 -85q49 -25 89 -32q2 0 18 -5t44 -13t43 -12q30 -8 50 -18q5 -45 5 -50q0 -10 -3 -29q-14 -1 -34 -1q-110 0 -187 10q-72 8 -238 8q-88 0 -233 -14q-48 -4 -70 -4q-2 22 -2 26l-1 26v9q21 33 79 49 q139 38 159 50q9 21 12 56q8 192 6 433l-5 428q-1 62 -0.5 118.5t0.5 102.5t-2 57t-6 15q-6 5 -14 6q-38 6 -148 6q-43 0 -100 -13.5t-73 -24.5q-13 -9 -22 -33t-22 -75t-24 -84q-6 -19 -19.5 -32t-20.5 -13q-44 27 -56 44v297v86zM1744 128q33 0 42 -18.5t-11 -44.5 l-126 -162q-20 -26 -49 -26t-49 26l-126 162q-20 26 -11 44.5t42 18.5h80v1024h-80q-33 0 -42 18.5t11 44.5l126 162q20 26 49 26t49 -26l126 -162q20 -26 11 -44.5t-42 -18.5h-80v-1024h80z" />
-<glyph unicode="&#xf035;" d="M81 1407l54 -27q20 -5 211 -5h130l19 3l115 1l446 -1h318l34 -2q14 -1 28 7t21 16l7 8l42 1q15 0 28 -1v-104.5t1 -131.5l1 -100l-1 -58q0 -32 -4 -51q-39 -15 -68 -18q-25 43 -54 128q-8 24 -15.5 62.5t-11.5 65.5t-6 29q-13 15 -27 19q-7 2 -58.5 2t-138.5 -1t-128 -1 q-94 0 -127 -5q-10 -97 -8 -136l1 -152v52l3 -359l-1 -147q-1 -46 11 -85q49 -25 89 -32q2 0 18 -5t44 -13t43 -12q30 -8 50 -18q5 -45 5 -50q0 -10 -3 -29q-14 -1 -34 -1q-110 0 -187 10q-72 8 -238 8q-82 0 -233 -13q-45 -5 -70 -5q-2 22 -2 26l-1 26v9q21 33 79 49 q139 38 159 50q9 21 12 56q6 137 6 433l-5 44q0 265 -2 278q-2 11 -6 15q-6 5 -14 6q-38 6 -148 6q-50 0 -168.5 -14t-132.5 -24q-13 -9 -22 -33t-22 -75t-24 -84q-6 -19 -19.5 -32t-20.5 -13q-44 27 -56 44v297v86zM1505 113q26 -20 26 -49t-26 -49l-162 -126 q-26 -20 -44.5 -11t-18.5 42v80h-1024v-80q0 -33 -18.5 -42t-44.5 11l-162 126q-26 20 -26 49t26 49l162 126q26 20 44.5 11t18.5 -42v-80h1024v80q0 33 18.5 42t44.5 -11z" />
-<glyph unicode="&#xf036;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf037;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h896q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45t-45 -19 h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h640q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf038;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf039;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf03a;" horiz-adv-x="1792" d="M256 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM256 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5 t9.5 -22.5zM256 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344 q13 0 22.5 -9.5t9.5 -22.5zM256 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5 t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t
 -22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192 q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03b;" horiz-adv-x="1792" d="M384 992v-576q0 -13 -9.5 -22.5t-22.5 -9.5q-14 0 -23 9l-288 288q-9 9 -9 23t9 23l288 288q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03c;" horiz-adv-x="1792" d="M352 704q0 -14 -9 -23l-288 -288q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v576q0 13 9.5 22.5t22.5 9.5q14 0 23 -9l288 -288q9 -9 9 -23zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03d;" horiz-adv-x="1792" d="M1792 1184v-1088q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-403 403v-166q0 -119 -84.5 -203.5t-203.5 -84.5h-704q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h704q119 0 203.5 -84.5t84.5 -203.5v-165l403 402q18 19 45 19q12 0 25 -5 q39 -17 39 -59z" />
-<glyph unicode="&#xf03e;" horiz-adv-x="1920" d="M640 960q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 576v-448h-1408v192l320 320l160 -160l512 512zM1760 1280h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5v1216 q0 13 -9.5 22.5t-22.5 9.5zM1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf040;" d="M363 0l91 91l-235 235l-91 -91v-107h128v-128h107zM886 928q0 22 -22 22q-10 0 -17 -7l-542 -542q-7 -7 -7 -17q0 -22 22 -22q10 0 17 7l542 542q7 7 7 17zM832 1120l416 -416l-832 -832h-416v416zM1515 1024q0 -53 -37 -90l-166 -166l-416 416l166 165q36 38 90 38 q53 0 91 -38l235 -234q37 -39 37 -91z" />
-<glyph unicode="&#xf041;" horiz-adv-x="1024" d="M768 896q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1024 896q0 -109 -33 -179l-364 -774q-16 -33 -47.5 -52t-67.5 -19t-67.5 19t-46.5 52l-365 774q-33 70 -33 179q0 212 150 362t362 150t362 -150t150 -362z" />
-<glyph unicode="&#xf042;" d="M768 96v1088q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf043;" horiz-adv-x="1024" d="M512 384q0 36 -20 69q-1 1 -15.5 22.5t-25.5 38t-25 44t-21 50.5q-4 16 -21 16t-21 -16q-7 -23 -21 -50.5t-25 -44t-25.5 -38t-15.5 -22.5q-20 -33 -20 -69q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 512q0 -212 -150 -362t-362 -150t-362 150t-150 362 q0 145 81 275q6 9 62.5 90.5t101 151t99.5 178t83 201.5q9 30 34 47t51 17t51.5 -17t33.5 -47q28 -93 83 -201.5t99.5 -178t101 -151t62.5 -90.5q81 -127 81 -275z" />
-<glyph unicode="&#xf044;" horiz-adv-x="1792" d="M888 352l116 116l-152 152l-116 -116v-56h96v-96h56zM1328 1072q-16 16 -33 -1l-350 -350q-17 -17 -1 -33t33 1l350 350q17 17 1 33zM1408 478v-190q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-14 -14 -32 -8q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v126q0 13 9 22l64 64q15 15 35 7t20 -29zM1312 1216l288 -288l-672 -672h-288v288zM1756 1084l-92 -92 l-288 288l92 92q28 28 68 28t68 -28l152 -152q28 -28 28 -68t-28 -68z" />
-<glyph unicode="&#xf045;" horiz-adv-x="1664" d="M1408 547v-259q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h255v0q13 0 22.5 -9.5t9.5 -22.5q0 -27 -26 -32q-77 -26 -133 -60q-10 -4 -16 -4h-112q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832 q66 0 113 47t47 113v214q0 19 18 29q28 13 54 37q16 16 35 8q21 -9 21 -29zM1645 1043l-384 -384q-18 -19 -45 -19q-12 0 -25 5q-39 17 -39 59v192h-160q-323 0 -438 -131q-119 -137 -74 -473q3 -23 -20 -34q-8 -2 -12 -2q-16 0 -26 13q-10 14 -21 31t-39.5 68.5t-49.5 99.5 t-38.5 114t-17.5 122q0 49 3.5 91t14 90t28 88t47 81.5t68.5 74t94.5 61.5t124.5 48.5t159.5 30.5t196.5 11h160v192q0 42 39 59q13 5 25 5q26 0 45 -19l384 -384q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf046;" horiz-adv-x="1664" d="M1408 606v-318q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-10 -10 -23 -10q-3 0 -9 2q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832 q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v254q0 13 9 22l64 64q10 10 23 10q6 0 12 -3q20 -8 20 -29zM1639 1095l-814 -814q-24 -24 -57 -24t-57 24l-430 430q-24 24 -24 57t24 57l110 110q24 24 57 24t57 -24l263 -263l647 647q24 24 57 24t57 -24l110 -110 q24 -24 24 -57t-24 -57z" />
-<glyph unicode="&#xf047;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-384v-384h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v384h-384v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45 t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h384v384h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45t-19 -45t-45 -19h-128v-384h384v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf048;" horiz-adv-x="1024" d="M979 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19z" />
-<glyph unicode="&#xf049;" horiz-adv-x="1792" d="M1747 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19l710 710 q19 19 32 13t13 -32v-710q4 11 13 19z" />
-<glyph unicode="&#xf04a;" horiz-adv-x="1664" d="M1619 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-8 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-19 19 -19 45t19 45l710 710q19 19 32 13t13 -32v-710q5 11 13 19z" />
-<glyph unicode="&#xf04b;" horiz-adv-x="1408" d="M1384 609l-1328 -738q-23 -13 -39.5 -3t-16.5 36v1472q0 26 16.5 36t39.5 -3l1328 -738q23 -13 23 -31t-23 -31z" />
-<glyph unicode="&#xf04c;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45zM640 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf04d;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf04e;" horiz-adv-x="1664" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q19 -19 19 -45t-19 -45l-710 -710q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
-<glyph unicode="&#xf050;" horiz-adv-x="1792" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19l-710 -710 q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
-<glyph unicode="&#xf051;" horiz-adv-x="1024" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19z" />
-<glyph unicode="&#xf052;" horiz-adv-x="1538" d="M14 557l710 710q19 19 45 19t45 -19l710 -710q19 -19 13 -32t-32 -13h-1472q-26 0 -32 13t13 32zM1473 0h-1408q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1408q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19z" />
-<glyph unicode="&#xf053;" horiz-adv-x="1152" d="M742 -37l-652 651q-37 37 -37 90.5t37 90.5l652 651q37 37 90.5 37t90.5 -37l75 -75q37 -37 37 -90.5t-37 -90.5l-486 -486l486 -485q37 -38 37 -91t-37 -90l-75 -75q-37 -37 -90.5 -37t-90.5 37z" />
-<glyph unicode="&#xf054;" horiz-adv-x="1152" d="M1099 704q0 -52 -37 -91l-652 -651q-37 -37 -90 -37t-90 37l-76 75q-37 39 -37 91q0 53 37 90l486 486l-486 485q-37 39 -37 91q0 53 37 90l76 75q36 38 90 38t90 -38l652 -651q37 -37 37 -90z" />
-<glyph unicode="&#xf055;" d="M1216 576v128q0 26 -19 45t-45 19h-256v256q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-256h-256q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h256v-256q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v256h256q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5 t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf056;" d="M1216 576v128q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 t103 -385.5z" />
-<glyph unicode="&#xf057;" d="M1149 414q0 26 -19 45l-181 181l181 181q19 19 19 45q0 27 -19 46l-90 90q-19 19 -46 19q-26 0 -45 -19l-181 -181l-181 181q-19 19 -45 19q-27 0 -46 -19l-90 -90q-19 -19 -19 -46q0 -26 19 -45l181 -181l-181 -181q-19 -19 -19 -45q0 -27 19 -46l90 -90q19 -19 46 -19 q26 0 45 19l181 181l181 -181q19 -19 45 -19q27 0 46 19l90 90q19 19 19 46zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf058;" d="M1284 802q0 28 -18 46l-91 90q-19 19 -45 19t-45 -19l-408 -407l-226 226q-19 19 -45 19t-45 -19l-91 -90q-18 -18 -18 -46q0 -27 18 -45l362 -362q19 -19 45 -19q27 0 46 19l543 543q18 18 18 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf059;" d="M896 160v192q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h192q14 0 23 9t9 23zM1152 832q0 88 -55.5 163t-138.5 116t-170 41q-243 0 -371 -213q-15 -24 8 -42l132 -100q7 -6 19 -6q16 0 25 12q53 68 86 92q34 24 86 24q48 0 85.5 -26t37.5 -59 q0 -38 -20 -61t-68 -45q-63 -28 -115.5 -86.5t-52.5 -125.5v-36q0 -14 9 -23t23 -9h192q14 0 23 9t9 23q0 19 21.5 49.5t54.5 49.5q32 18 49 28.5t46 35t44.5 48t28 60.5t12.5 81zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05a;" d="M1024 160v160q0 14 -9 23t-23 9h-96v512q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h96v-320h-96q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h448q14 0 23 9t9 23zM896 1056v160q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23 t23 -9h192q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05b;" d="M1197 512h-109q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h109q-32 108 -112.5 188.5t-188.5 112.5v-109q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v109q-108 -32 -188.5 -112.5t-112.5 -188.5h109q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-109 q32 -108 112.5 -188.5t188.5 -112.5v109q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-109q108 32 188.5 112.5t112.5 188.5zM1536 704v-128q0 -26 -19 -45t-45 -19h-143q-37 -161 -154.5 -278.5t-278.5 -154.5v-143q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v143 q-161 37 -278.5 154.5t-154.5 278.5h-143q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h143q37 161 154.5 278.5t278.5 154.5v143q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-143q161 -37 278.5 -154.5t154.5 -278.5h143q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf05c;" d="M1097 457l-146 -146q-10 -10 -23 -10t-23 10l-137 137l-137 -137q-10 -10 -23 -10t-23 10l-146 146q-10 10 -10 23t10 23l137 137l-137 137q-10 10 -10 23t10 23l146 146q10 10 23 10t23 -10l137 -137l137 137q10 10 23 10t23 -10l146 -146q10 -10 10 -23t-10 -23 l-137 -137l137 -137q10 -10 10 -23t-10 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5 t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05d;" d="M1171 723l-422 -422q-19 -19 -45 -19t-45 19l-294 294q-19 19 -19 45t19 45l102 102q19 19 45 19t45 -19l147 -147l275 275q19 19 45 19t45 -19l102 -102q19 -19 19 -45t-19 -45zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198 t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05e;" d="M1312 643q0 161 -87 295l-754 -753q137 -89 297 -89q111 0 211.5 43.5t173.5 116.5t116 174.5t43 212.5zM313 344l755 754q-135 91 -300 91q-148 0 -273 -73t-198 -199t-73 -274q0 -162 89 -299zM1536 643q0 -157 -61 -300t-163.5 -246t-245 -164t-298.5 -61t-298.5 61 t-245 164t-163.5 246t-61 300t61 299.5t163.5 245.5t245 164t298.5 61t298.5 -61t245 -164t163.5 -245.5t61 -299.5z" />
-<glyph unicode="&#xf060;" d="M1536 640v-128q0 -53 -32.5 -90.5t-84.5 -37.5h-704l293 -294q38 -36 38 -90t-38 -90l-75 -76q-37 -37 -90 -37q-52 0 -91 37l-651 652q-37 37 -37 90q0 52 37 91l651 650q38 38 91 38q52 0 90 -38l75 -74q38 -38 38 -91t-38 -91l-293 -293h704q52 0 84.5 -37.5 t32.5 -90.5z" />
-<glyph unicode="&#xf061;" d="M1472 576q0 -54 -37 -91l-651 -651q-39 -37 -91 -37q-51 0 -90 37l-75 75q-38 38 -38 91t38 91l293 293h-704q-52 0 -84.5 37.5t-32.5 90.5v128q0 53 32.5 90.5t84.5 37.5h704l-293 294q-38 36 -38 90t38 90l75 75q38 38 90 38q53 0 91 -38l651 -651q37 -35 37 -90z" />
-<glyph unicode="&#xf062;" horiz-adv-x="1664" d="M1611 565q0 -51 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-294 293v-704q0 -52 -37.5 -84.5t-90.5 -32.5h-128q-53 0 -90.5 32.5t-37.5 84.5v704l-294 -293q-36 -38 -90 -38t-90 38l-75 75q-38 38 -38 90q0 53 38 91l651 651q35 37 90 37q54 0 91 -37l651 -651 q37 -39 37 -91z" />
-<glyph unicode="&#xf063;" horiz-adv-x="1664" d="M1611 704q0 -53 -37 -90l-651 -652q-39 -37 -91 -37q-53 0 -90 37l-651 652q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l294 -294v704q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-704l294 294q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
-<glyph unicode="&#xf064;" horiz-adv-x="1792" d="M1792 896q0 -26 -19 -45l-512 -512q-19 -19 -45 -19t-45 19t-19 45v256h-224q-98 0 -175.5 -6t-154 -21.5t-133 -42.5t-105.5 -69.5t-80 -101t-48.5 -138.5t-17.5 -181q0 -55 5 -123q0 -6 2.5 -23.5t2.5 -26.5q0 -15 -8.5 -25t-23.5 -10q-16 0 -28 17q-7 9 -13 22 t-13.5 30t-10.5 24q-127 285 -127 451q0 199 53 333q162 403 875 403h224v256q0 26 19 45t45 19t45 -19l512 -512q19 -19 19 -45z" />
-<glyph unicode="&#xf065;" d="M755 480q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23zM1536 1344v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332 q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf066;" d="M768 576v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45zM1523 1248q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45 t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23z" />
-<glyph unicode="&#xf067;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-416v-416q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v416h-416q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h416v416q0 40 28 68t68 28h192q40 0 68 -28t28 -68v-416h416q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf068;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-1216q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h1216q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf069;" horiz-adv-x="1664" d="M1482 486q46 -26 59.5 -77.5t-12.5 -97.5l-64 -110q-26 -46 -77.5 -59.5t-97.5 12.5l-266 153v-307q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v307l-266 -153q-46 -26 -97.5 -12.5t-77.5 59.5l-64 110q-26 46 -12.5 97.5t59.5 77.5l266 154l-266 154 q-46 26 -59.5 77.5t12.5 97.5l64 110q26 46 77.5 59.5t97.5 -12.5l266 -153v307q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-307l266 153q46 26 97.5 12.5t77.5 -59.5l64 -110q26 -46 12.5 -97.5t-59.5 -77.5l-266 -154z" />
-<glyph unicode="&#xf06a;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM896 161v190q0 14 -9 23.5t-22 9.5h-192q-13 0 -23 -10t-10 -23v-190q0 -13 10 -23t23 -10h192 q13 0 22 9.5t9 23.5zM894 505l18 621q0 12 -10 18q-10 8 -24 8h-220q-14 0 -24 -8q-10 -6 -10 -18l17 -621q0 -10 10 -17.5t24 -7.5h185q14 0 23.5 7.5t10.5 17.5z" />
-<glyph unicode="&#xf06b;" d="M928 180v56v468v192h-320v-192v-468v-56q0 -25 18 -38.5t46 -13.5h192q28 0 46 13.5t18 38.5zM472 1024h195l-126 161q-26 31 -69 31q-40 0 -68 -28t-28 -68t28 -68t68 -28zM1160 1120q0 40 -28 68t-68 28q-43 0 -69 -31l-125 -161h194q40 0 68 28t28 68zM1536 864v-320 q0 -14 -9 -23t-23 -9h-96v-416q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v416h-96q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h440q-93 0 -158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5q107 0 168 -77l128 -165l128 165q61 77 168 77q93 0 158.5 -65.5t65.5 -158.5 t-65.5 -158.5t-158.5 -65.5h440q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf06c;" horiz-adv-x="1792" d="M1280 832q0 26 -19 45t-45 19q-172 0 -318 -49.5t-259.5 -134t-235.5 -219.5q-19 -21 -19 -45q0 -26 19 -45t45 -19q24 0 45 19q27 24 74 71t67 66q137 124 268.5 176t313.5 52q26 0 45 19t19 45zM1792 1030q0 -95 -20 -193q-46 -224 -184.5 -383t-357.5 -268 q-214 -108 -438 -108q-148 0 -286 47q-15 5 -88 42t-96 37q-16 0 -39.5 -32t-45 -70t-52.5 -70t-60 -32q-30 0 -51 11t-31 24t-27 42q-2 4 -6 11t-5.5 10t-3 9.5t-1.5 13.5q0 35 31 73.5t68 65.5t68 56t31 48q0 4 -14 38t-16 44q-9 51 -9 104q0 115 43.5 220t119 184.5 t170.5 139t204 95.5q55 18 145 25.5t179.5 9t178.5 6t163.5 24t113.5 56.5l29.5 29.5t29.5 28t27 20t36.5 16t43.5 4.5q39 0 70.5 -46t47.5 -112t24 -124t8 -96z" />
-<glyph unicode="&#xf06d;" horiz-adv-x="1408" d="M1408 -160v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1152 896q0 -78 -24.5 -144t-64 -112.5t-87.5 -88t-96 -77.5t-87.5 -72t-64 -81.5t-24.5 -96.5q0 -96 67 -224l-4 1l1 -1 q-90 41 -160 83t-138.5 100t-113.5 122.5t-72.5 150.5t-27.5 184q0 78 24.5 144t64 112.5t87.5 88t96 77.5t87.5 72t64 81.5t24.5 96.5q0 94 -66 224l3 -1l-1 1q90 -41 160 -83t138.5 -100t113.5 -122.5t72.5 -150.5t27.5 -184z" />
-<glyph unicode="&#xf06e;" horiz-adv-x="1792" d="M1664 576q-152 236 -381 353q61 -104 61 -225q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 121 61 225q-229 -117 -381 -353q133 -205 333.5 -326.5t434.5 -121.5t434.5 121.5t333.5 326.5zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5 t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1792 576q0 -34 -20 -69q-140 -230 -376.5 -368.5t-499.5 -138.5t-499.5 139t-376.5 368q-20 35 -20 69t20 69q140 229 376.5 368t499.5 139t499.5 -139t376.5 -368q20 -35 20 -69z" />
-<glyph unicode="&#xf070;" horiz-adv-x="1792" d="M555 201l78 141q-87 63 -136 159t-49 203q0 121 61 225q-229 -117 -381 -353q167 -258 427 -375zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1307 1151q0 -7 -1 -9 q-105 -188 -315 -566t-316 -567l-49 -89q-10 -16 -28 -16q-12 0 -134 70q-16 10 -16 28q0 12 44 87q-143 65 -263.5 173t-208.5 245q-20 31 -20 69t20 69q153 235 380 371t496 136q89 0 180 -17l54 97q10 16 28 16q5 0 18 -6t31 -15.5t33 -18.5t31.5 -18.5t19.5 -11.5 q16 -10 16 -27zM1344 704q0 -139 -79 -253.5t-209 -164.5l280 502q8 -45 8 -84zM1792 576q0 -35 -20 -69q-39 -64 -109 -145q-150 -172 -347.5 -267t-419.5 -95l74 132q212 18 392.5 137t301.5 307q-115 179 -282 294l63 112q95 -64 182.5 -153t144.5 -184q20 -34 20 -69z " />
-<glyph unicode="&#xf071;" horiz-adv-x="1792" d="M1024 161v190q0 14 -9.5 23.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -23.5v-190q0 -14 9.5 -23.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 23.5zM1022 535l18 459q0 12 -10 19q-13 11 -24 11h-220q-11 0 -24 -11q-10 -7 -10 -21l17 -457q0 -10 10 -16.5t24 -6.5h185 q14 0 23.5 6.5t10.5 16.5zM1008 1469l768 -1408q35 -63 -2 -126q-17 -29 -46.5 -46t-63.5 -17h-1536q-34 0 -63.5 17t-46.5 46q-37 63 -2 126l768 1408q17 31 47 49t65 18t65 -18t47 -49z" />
-<glyph unicode="&#xf072;" horiz-adv-x="1408" d="M1376 1376q44 -52 12 -148t-108 -172l-161 -161l160 -696q5 -19 -12 -33l-128 -96q-7 -6 -19 -6q-4 0 -7 1q-15 3 -21 16l-279 508l-259 -259l53 -194q5 -17 -8 -31l-96 -96q-9 -9 -23 -9h-2q-15 2 -24 13l-189 252l-252 189q-11 7 -13 23q-1 13 9 25l96 97q9 9 23 9 q6 0 8 -1l194 -53l259 259l-508 279q-14 8 -17 24q-2 16 9 27l128 128q14 13 30 8l665 -159l160 160q76 76 172 108t148 -12z" />
-<glyph unicode="&#xf073;" horiz-adv-x="1664" d="M128 -128h288v288h-288v-288zM480 -128h320v288h-320v-288zM128 224h288v320h-288v-320zM480 224h320v320h-320v-320zM128 608h288v288h-288v-288zM864 -128h320v288h-320v-288zM480 608h320v288h-320v-288zM1248 -128h288v288h-288v-288zM864 224h320v320h-320v-320z M512 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1248 224h288v320h-288v-320zM864 608h320v288h-320v-288zM1248 608h288v288h-288v-288zM1280 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64 q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1664 1152v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47 h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" />
-<glyph unicode="&#xf074;" horiz-adv-x="1792" d="M666 1055q-60 -92 -137 -273q-22 45 -37 72.5t-40.5 63.5t-51 56.5t-63 35t-81.5 14.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q250 0 410 -225zM1792 256q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192q-32 0 -85 -0.5t-81 -1t-73 1 t-71 5t-64 10.5t-63 18.5t-58 28.5t-59 40t-55 53.5t-56 69.5q59 93 136 273q22 -45 37 -72.5t40.5 -63.5t51 -56.5t63 -35t81.5 -14.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1792 1152q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5 v192h-256q-48 0 -87 -15t-69 -45t-51 -61.5t-45 -77.5q-32 -62 -78 -171q-29 -66 -49.5 -111t-54 -105t-64 -100t-74 -83t-90 -68.5t-106.5 -42t-128 -16.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q48 0 87 15t69 45t51 61.5t45 77.5q32 62 78 171q29 66 49.5 111 t54 105t64 100t74 83t90 68.5t106.5 42t128 16.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" />
-<glyph unicode="&#xf075;" horiz-adv-x="1792" d="M1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22q-17 -2 -30.5 9t-17.5 29v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281 q0 130 71 248.5t191 204.5t286 136.5t348 50.5q244 0 450 -85.5t326 -233t120 -321.5z" />
-<glyph unicode="&#xf076;" d="M1536 704v-128q0 -201 -98.5 -362t-274 -251.5t-395.5 -90.5t-395.5 90.5t-274 251.5t-98.5 362v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-128q0 -52 23.5 -90t53.5 -57t71 -30t64 -13t44 -2t44 2t64 13t71 30t53.5 57t23.5 90v128q0 26 19 45t45 19h384 q26 0 45 -19t19 -45zM512 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45zM1536 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf077;" horiz-adv-x="1664" d="M1611 320q0 -53 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-486 485l-486 -485q-36 -38 -90 -38t-90 38l-75 75q-38 36 -38 90q0 53 38 91l651 651q37 37 90 37q52 0 91 -37l650 -651q38 -38 38 -91z" />
-<glyph unicode="&#xf078;" horiz-adv-x="1664" d="M1611 832q0 -53 -37 -90l-651 -651q-38 -38 -91 -38q-54 0 -90 38l-651 651q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l486 -486l486 486q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
-<glyph unicode="&#xf079;" horiz-adv-x="1920" d="M1280 32q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-8 0 -13.5 2t-9 7t-5.5 8t-3 11.5t-1 11.5v13v11v160v416h-192q-26 0 -45 19t-19 45q0 24 15 41l320 384q19 22 49 22t49 -22l320 -384q15 -17 15 -41q0 -26 -19 -45t-45 -19h-192v-384h576q16 0 25 -11l160 -192q7 -11 7 -21 zM1920 448q0 -24 -15 -41l-320 -384q-20 -23 -49 -23t-49 23l-320 384q-15 17 -15 41q0 26 19 45t45 19h192v384h-576q-16 0 -25 12l-160 192q-7 9 -7 20q0 13 9.5 22.5t22.5 9.5h960q8 0 13.5 -2t9 -7t5.5 -8t3 -11.5t1 -11.5v-13v-11v-160v-416h192q26 0 45 -19t19 -45z " />
-<glyph unicode="&#xf07a;" horiz-adv-x="1664" d="M640 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1536 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1664 1088v-512q0 -24 -16 -42.5t-41 -21.5 l-1044 -122q1 -7 4.5 -21.5t6 -26.5t2.5 -22q0 -16 -24 -64h920q26 0 45 -19t19 -45t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 14 11 39.5t29.5 59.5t20.5 38l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t20 -15.5t13 -24.5t7.5 -26.5 t5.5 -29.5t4.5 -25.5h1201q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf07b;" horiz-adv-x="1664" d="M1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
-<glyph unicode="&#xf07c;" horiz-adv-x="1920" d="M1879 584q0 -31 -31 -66l-336 -396q-43 -51 -120.5 -86.5t-143.5 -35.5h-1088q-34 0 -60.5 13t-26.5 43q0 31 31 66l336 396q43 51 120.5 86.5t143.5 35.5h1088q34 0 60.5 -13t26.5 -43zM1536 928v-160h-832q-94 0 -197 -47.5t-164 -119.5l-337 -396l-5 -6q0 4 -0.5 12.5 t-0.5 12.5v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158z" />
-<glyph unicode="&#xf07d;" horiz-adv-x="768" d="M704 1216q0 -26 -19 -45t-45 -19h-128v-1024h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v1024h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf07e;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-1024v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h1024v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf080;" horiz-adv-x="1920" d="M512 512v-384h-256v384h256zM896 1024v-896h-256v896h256zM1280 768v-640h-256v640h256zM1664 1152v-1024h-256v1024h256zM1792 32v1216q0 13 -9.5 22.5t-22.5 9.5h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5z M1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf081;" d="M1280 926q-56 -25 -121 -34q68 40 93 117q-65 -38 -134 -51q-61 66 -153 66q-87 0 -148.5 -61.5t-61.5 -148.5q0 -29 5 -48q-129 7 -242 65t-192 155q-29 -50 -29 -106q0 -114 91 -175q-47 1 -100 26v-2q0 -75 50 -133.5t123 -72.5q-29 -8 -51 -8q-13 0 -39 4 q21 -63 74.5 -104t121.5 -42q-116 -90 -261 -90q-26 0 -50 3q148 -94 322 -94q112 0 210 35.5t168 95t120.5 137t75 162t24.5 168.5q0 18 -1 27q63 45 105 109zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5 t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf082;" d="M1307 618l23 219h-198v109q0 49 15.5 68.5t71.5 19.5h110v219h-175q-152 0 -218 -72t-66 -213v-131h-131v-219h131v-635h262v635h175zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960 q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf083;" horiz-adv-x="1792" d="M928 704q0 14 -9 23t-23 9q-66 0 -113 -47t-47 -113q0 -14 9 -23t23 -9t23 9t9 23q0 40 28 68t68 28q14 0 23 9t9 23zM1152 574q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM128 0h1536v128h-1536v-128zM1280 574q0 159 -112.5 271.5 t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM256 1216h384v128h-384v-128zM128 1024h1536v118v138h-828l-64 -128h-644v-128zM1792 1280v-1280q0 -53 -37.5 -90.5t-90.5 -37.5h-1536q-53 0 -90.5 37.5t-37.5 90.5v1280 q0 53 37.5 90.5t90.5 37.5h1536q53 0 90.5 -37.5t37.5 -90.5z" />
-<glyph unicode="&#xf084;" horiz-adv-x="1792" d="M832 1024q0 80 -56 136t-136 56t-136 -56t-56 -136q0 -42 19 -83q-41 19 -83 19q-80 0 -136 -56t-56 -136t56 -136t136 -56t136 56t56 136q0 42 -19 83q41 -19 83 -19q80 0 136 56t56 136zM1683 320q0 -17 -49 -66t-66 -49q-9 0 -28.5 16t-36.5 33t-38.5 40t-24.5 26 l-96 -96l220 -220q28 -28 28 -68q0 -42 -39 -81t-81 -39q-40 0 -68 28l-671 671q-176 -131 -365 -131q-163 0 -265.5 102.5t-102.5 265.5q0 160 95 313t248 248t313 95q163 0 265.5 -102.5t102.5 -265.5q0 -189 -131 -365l355 -355l96 96q-3 3 -26 24.5t-40 38.5t-33 36.5 t-16 28.5q0 17 49 66t66 49q13 0 23 -10q6 -6 46 -44.5t82 -79.5t86.5 -86t73 -78t28.5 -41z" />
-<glyph unicode="&#xf085;" horiz-adv-x="1920" d="M896 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1664 128q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1152q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1280 731v-185q0 -10 -7 -19.5t-16 -10.5l-155 -24q-11 -35 -32 -76q34 -48 90 -115q7 -10 7 -20q0 -12 -7 -19q-23 -30 -82.5 -89.5t-78.5 -59.5q-11 0 -21 7l-115 90q-37 -19 -77 -31q-11 -108 -23 -155q-7 -24 -30 -24h-186q-11 0 -20 7.5t-10 17.5 l-23 153q-34 10 -75 31l-118 -89q-7 -7 -20 -7q-11 0 -21 8q-144 133 -144 160q0 9 7 19q10 14 41 53t47 61q-23 44 -35 82l-152 24q-10 1 -17 9.5t-7 19.5v185q0 10 7 19.5t16 10.5l155 24q11 35 32 76q-34 48 -90 115q-7 11 -7 20q0 12 7 20q22 30 82 89t79 59q11 0 21 -7 l115 -90q34 18 77 32q11 108 23 154q7 24 30 24h186q11 0 20 -7.5t10 -17.5l23 -153q34 -10 75 -31l118 89q8 7 20 7q11 0 21 -8q144 -133 144 -160q0 -9 -7 -19q-12 -16 -42 -54t-45 -60q23 -48 34 -82l152 
 -23q10 -2 17 -10.5t7 -19.5zM1920 198v-140q0 -16 -149 -31 q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20 t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31zM1920 1222v-140q0 -16 -149 -31q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68 q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70 q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31z" />
-<glyph unicode="&#xf086;" horiz-adv-x="1792" d="M1408 768q0 -139 -94 -257t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q-124 72 -195 177t-71 224 q0 139 94 257t256.5 186.5t353.5 68.5t353.5 -68.5t256.5 -186.5t94 -257zM1792 512q0 -120 -71 -224.5t-195 -176.5q10 -24 20.5 -44t25 -38.5t22.5 -29t26 -29.5t23 -25q1 -1 4 -4.5t4.5 -5t4 -5t3.5 -5.5l2.5 -5t2 -6t0.5 -6.5t-1 -6.5q-3 -14 -13 -22t-22 -7 q-50 7 -86 16q-154 40 -278 128q-90 -16 -176 -16q-271 0 -472 132q58 -4 88 -4q161 0 309 45t264 129q125 92 192 212t67 254q0 77 -23 152q129 -71 204 -178t75 -230z" />
-<glyph unicode="&#xf087;" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 768q0 51 -39 89.5t-89 38.5h-352q0 58 48 159.5t48 160.5q0 98 -32 145t-128 47q-26 -26 -38 -85t-30.5 -125.5t-59.5 -109.5q-22 -23 -77 -91q-4 -5 -23 -30t-31.5 -41t-34.5 -42.5 t-40 -44t-38.5 -35.5t-40 -27t-35.5 -9h-32v-640h32q13 0 31.5 -3t33 -6.5t38 -11t35 -11.5t35.5 -12.5t29 -10.5q211 -73 342 -73h121q192 0 192 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5q32 1 53.5 47t21.5 81zM1536 769 q0 -89 -49 -163q9 -33 9 -69q0 -77 -38 -144q3 -21 3 -43q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5h-36h-93q-96 0 -189.5 22.5t-216.5 65.5q-116 40 -138 40h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h274q36 24 137 155q58 75 107 128 q24 25 35.5 85.5t30.5 126.5t62 108q39 37 90 37q84 0 151 -32.5t102 -101.5t35 -186q0 -93 -48 -192h176q104 0 180 -76t76 -179z" />
-<glyph unicode="&#xf088;" d="M256 1088q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 512q0 35 -21.5 81t-53.5 47q15 17 25 47.5t10 55.5q0 69 -53 119q18 32 18 69t-17.5 73.5t-47.5 52.5q5 30 5 56q0 85 -49 126t-136 41h-128q-131 0 -342 -73q-5 -2 -29 -10.5 t-35.5 -12.5t-35 -11.5t-38 -11t-33 -6.5t-31.5 -3h-32v-640h32q16 0 35.5 -9t40 -27t38.5 -35.5t40 -44t34.5 -42.5t31.5 -41t23 -30q55 -68 77 -91q41 -43 59.5 -109.5t30.5 -125.5t38 -85q96 0 128 47t32 145q0 59 -48 160.5t-48 159.5h352q50 0 89 38.5t39 89.5z M1536 511q0 -103 -76 -179t-180 -76h-176q48 -99 48 -192q0 -118 -35 -186q-35 -69 -102 -101.5t-151 -32.5q-51 0 -90 37q-34 33 -54 82t-25.5 90.5t-17.5 84.5t-31 64q-48 50 -107 127q-101 131 -137 155h-274q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5 h288q22 0 138 40q128 44 223 66t200 22h112q140 0 226.5 -79t85.5 -216v-5q60 -77 60 -178q0 -22 -3 -43q38 -67 38 -144q0 -36 -9 -69q49 -74 49 -163z" />
-<glyph unicode="&#xf089;" horiz-adv-x="896" d="M832 1504v-1339l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41z" />
-<glyph unicode="&#xf08a;" horiz-adv-x="1792" d="M1664 940q0 81 -21.5 143t-55 98.5t-81.5 59.5t-94 31t-98 8t-112 -25.5t-110.5 -64t-86.5 -72t-60 -61.5q-18 -22 -49 -22t-49 22q-24 28 -60 61.5t-86.5 72t-110.5 64t-112 25.5t-98 -8t-94 -31t-81.5 -59.5t-55 -98.5t-21.5 -143q0 -168 187 -355l581 -560l580 559 q188 188 188 356zM1792 940q0 -221 -229 -450l-623 -600q-18 -18 -44 -18t-44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5 q224 0 351 -124t127 -344z" />
-<glyph unicode="&#xf08b;" horiz-adv-x="1664" d="M640 96q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h320q13 0 22.5 -9.5t9.5 -22.5q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-66 0 -113 -47t-47 -113v-704 q0 -66 47 -113t113 -47h288h11h13t11.5 -1t11.5 -3t8 -5.5t7 -9t2 -13.5zM1568 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45z" />
-<glyph unicode="&#xf08c;" d="M237 122h231v694h-231v-694zM483 1030q-1 52 -36 86t-93 34t-94.5 -34t-36.5 -86q0 -51 35.5 -85.5t92.5 -34.5h1q59 0 95 34.5t36 85.5zM1068 122h231v398q0 154 -73 233t-193 79q-136 0 -209 -117h2v101h-231q3 -66 0 -694h231v388q0 38 7 56q15 35 45 59.5t74 24.5 q116 0 116 -157v-371zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf08d;" horiz-adv-x="1152" d="M480 672v448q0 14 -9 23t-23 9t-23 -9t-9 -23v-448q0 -14 9 -23t23 -9t23 9t9 23zM1152 320q0 -26 -19 -45t-45 -19h-429l-51 -483q-2 -12 -10.5 -20.5t-20.5 -8.5h-1q-27 0 -32 27l-76 485h-404q-26 0 -45 19t-19 45q0 123 78.5 221.5t177.5 98.5v512q-52 0 -90 38 t-38 90t38 90t90 38h640q52 0 90 -38t38 -90t-38 -90t-90 -38v-512q99 0 177.5 -98.5t78.5 -221.5z" />
-<glyph unicode="&#xf08e;" horiz-adv-x="1792" d="M1408 608v-320q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v320 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1792 1472v-512q0 -26 -19 -45t-45 -19t-45 19l-176 176l-652 -652q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l652 652l-176 176q-19 19 -19 45t19 45t45 19h512q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf090;" d="M1184 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45zM1536 992v-704q0 -119 -84.5 -203.5t-203.5 -84.5h-320q-13 0 -22.5 9.5t-9.5 22.5 q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q66 0 113 47t47 113v704q0 66 -47 113t-113 47h-288h-11h-13t-11.5 1t-11.5 3t-8 5.5t-7 9t-2 13.5q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf091;" horiz-adv-x="1664" d="M458 653q-74 162 -74 371h-256v-96q0 -78 94.5 -162t235.5 -113zM1536 928v96h-256q0 -209 -74 -371q141 29 235.5 113t94.5 162zM1664 1056v-128q0 -71 -41.5 -143t-112 -130t-173 -97.5t-215.5 -44.5q-42 -54 -95 -95q-38 -34 -52.5 -72.5t-14.5 -89.5q0 -54 30.5 -91 t97.5 -37q75 0 133.5 -45.5t58.5 -114.5v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 69 58.5 114.5t133.5 45.5q67 0 97.5 37t30.5 91q0 51 -14.5 89.5t-52.5 72.5q-53 41 -95 95q-113 5 -215.5 44.5t-173 97.5t-112 130t-41.5 143v128q0 40 28 68t68 28h288v96 q0 66 47 113t113 47h576q66 0 113 -47t47 -113v-96h288q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf092;" d="M394 184q-8 -9 -20 3q-13 11 -4 19q8 9 20 -3q12 -11 4 -19zM352 245q9 -12 0 -19q-8 -6 -17 7t0 18q9 7 17 -6zM291 305q-5 -7 -13 -2q-10 5 -7 12q3 5 13 2q10 -5 7 -12zM322 271q-6 -7 -16 3q-9 11 -2 16q6 6 16 -3q9 -11 2 -16zM451 159q-4 -12 -19 -6q-17 4 -13 15 t19 7q16 -5 13 -16zM514 154q0 -11 -16 -11q-17 -2 -17 11q0 11 16 11q17 2 17 -11zM572 164q2 -10 -14 -14t-18 8t14 15q16 2 18 -9zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-224q-16 0 -24.5 1t-19.5 5t-16 14.5t-5 27.5v239q0 97 -52 142q57 6 102.5 18t94 39 t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103 q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -103t0.5 
 -68q0 -22 -11 -33.5t-22 -13t-33 -1.5 h-224q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf093;" horiz-adv-x="1664" d="M1280 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 288v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h427q21 -56 70.5 -92 t110.5 -36h256q61 0 110.5 36t70.5 92h427q40 0 68 -28t28 -68zM1339 936q-17 -40 -59 -40h-256v-448q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v448h-256q-42 0 -59 40q-17 39 14 69l448 448q18 19 45 19t45 -19l448 -448q31 -30 14 -69z" />
-<glyph unicode="&#xf094;" d="M1407 710q0 44 -7 113.5t-18 96.5q-12 30 -17 44t-9 36.5t-4 48.5q0 23 5 68.5t5 67.5q0 37 -10 55q-4 1 -13 1q-19 0 -58 -4.5t-59 -4.5q-60 0 -176 24t-175 24q-43 0 -94.5 -11.5t-85 -23.5t-89.5 -34q-137 -54 -202 -103q-96 -73 -159.5 -189.5t-88 -236t-24.5 -248.5 q0 -40 12.5 -120t12.5 -121q0 -23 -11 -66.5t-11 -65.5t12 -36.5t34 -14.5q24 0 72.5 11t73.5 11q57 0 169.5 -15.5t169.5 -15.5q181 0 284 36q129 45 235.5 152.5t166 245.5t59.5 275zM1535 712q0 -165 -70 -327.5t-196 -288t-281 -180.5q-124 -44 -326 -44 q-57 0 -170 14.5t-169 14.5q-24 0 -72.5 -14.5t-73.5 -14.5q-73 0 -123.5 55.5t-50.5 128.5q0 24 11 68t11 67q0 40 -12.5 120.5t-12.5 121.5q0 111 18 217.5t54.5 209.5t100.5 194t150 156q78 59 232 120q194 78 316 78q60 0 175.5 -24t173.5 -24q19 0 57 5t58 5 q81 0 118 -50.5t37 -134.5q0 -23 -5 -68t-5 -68q0 -10 1 -18.5t3 -17t4 -13.5t6.5 -16t6.5 -17q16 -40 25 -118.5t9 -136.5z" />
-<glyph unicode="&#xf095;" horiz-adv-x="1408" d="M1408 296q0 -27 -10 -70.5t-21 -68.5q-21 -50 -122 -106q-94 -51 -186 -51q-27 0 -52.5 3.5t-57.5 12.5t-47.5 14.5t-55.5 20.5t-49 18q-98 35 -175 83q-128 79 -264.5 215.5t-215.5 264.5q-48 77 -83 175q-3 9 -18 49t-20.5 55.5t-14.5 47.5t-12.5 57.5t-3.5 52.5 q0 92 51 186q56 101 106 122q25 11 68.5 21t70.5 10q14 0 21 -3q18 -6 53 -76q11 -19 30 -54t35 -63.5t31 -53.5q3 -4 17.5 -25t21.5 -35.5t7 -28.5q0 -20 -28.5 -50t-62 -55t-62 -53t-28.5 -46q0 -9 5 -22.5t8.5 -20.5t14 -24t11.5 -19q76 -137 174 -235t235 -174 q2 -1 19 -11.5t24 -14t20.5 -8.5t22.5 -5q18 0 46 28.5t53 62t55 62t50 28.5q14 0 28.5 -7t35.5 -21.5t25 -17.5q25 -15 53.5 -31t63.5 -35t54 -30q70 -35 76 -53q3 -7 3 -21z" />
-<glyph unicode="&#xf096;" horiz-adv-x="1408" d="M1120 1280h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v832q0 66 -47 113t-113 47zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf097;" horiz-adv-x="1280" d="M1152 1280h-1024v-1242l423 406l89 85l89 -85l423 -406v1242zM1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289 q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
-<glyph unicode="&#xf098;" d="M1280 343q0 11 -2 16q-3 8 -38.5 29.5t-88.5 49.5l-53 29q-5 3 -19 13t-25 15t-21 5q-18 0 -47 -32.5t-57 -65.5t-44 -33q-7 0 -16.5 3.5t-15.5 6.5t-17 9.5t-14 8.5q-99 55 -170.5 126.5t-126.5 170.5q-2 3 -8.5 14t-9.5 17t-6.5 15.5t-3.5 16.5q0 13 20.5 33.5t45 38.5 t45 39.5t20.5 36.5q0 10 -5 21t-15 25t-13 19q-3 6 -15 28.5t-25 45.5t-26.5 47.5t-25 40.5t-16.5 18t-16 2q-48 0 -101 -22q-46 -21 -80 -94.5t-34 -130.5q0 -16 2.5 -34t5 -30.5t9 -33t10 -29.5t12.5 -33t11 -30q60 -164 216.5 -320.5t320.5 -216.5q6 -2 30 -11t33 -12.5 t29.5 -10t33 -9t30.5 -5t34 -2.5q57 0 130.5 34t94.5 80q22 53 22 101zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf099;" horiz-adv-x="1664" d="M1620 1128q-67 -98 -162 -167q1 -14 1 -42q0 -130 -38 -259.5t-115.5 -248.5t-184.5 -210.5t-258 -146t-323 -54.5q-271 0 -496 145q35 -4 78 -4q225 0 401 138q-105 2 -188 64.5t-114 159.5q33 -5 61 -5q43 0 85 11q-112 23 -185.5 111.5t-73.5 205.5v4q68 -38 146 -41 q-66 44 -105 115t-39 154q0 88 44 163q121 -149 294.5 -238.5t371.5 -99.5q-8 38 -8 74q0 134 94.5 228.5t228.5 94.5q140 0 236 -102q109 21 205 78q-37 -115 -142 -178q93 10 186 50z" />
-<glyph unicode="&#xf09a;" horiz-adv-x="768" d="M511 980h257l-30 -284h-227v-824h-341v824h-170v284h170v171q0 182 86 275.5t283 93.5h227v-284h-142q-39 0 -62.5 -6.5t-34 -23.5t-13.5 -34.5t-3 -49.5v-142z" />
-<glyph unicode="&#xf09b;" d="M1536 640q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -39.5 7t-12.5 30v211q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5 q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23 q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -89t0.5 -54q0 -18 -13 -30t-40 -7q-232 77 -378.5 277.5t-146.5 451.5q0 209 103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf09c;" horiz-adv-x="1664" d="M1664 960v-256q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-192h96q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h672v192q0 185 131.5 316.5t316.5 131.5 t316.5 -131.5t131.5 -316.5z" />
-<glyph unicode="&#xf09d;" horiz-adv-x="1920" d="M1760 1408q66 0 113 -47t47 -113v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600zM160 1280q-13 0 -22.5 -9.5t-9.5 -22.5v-224h1664v224q0 13 -9.5 22.5t-22.5 9.5h-1600zM1760 0q13 0 22.5 9.5t9.5 22.5v608h-1664v-608 q0 -13 9.5 -22.5t22.5 -9.5h1600zM256 128v128h256v-128h-256zM640 128v128h384v-128h-384z" />
-<glyph unicode="&#xf09e;" horiz-adv-x="1408" d="M384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 69q2 -28 -17 -48q-18 -21 -47 -21h-135q-25 0 -43 16.5t-20 41.5q-22 229 -184.5 391.5t-391.5 184.5q-25 2 -41.5 20t-16.5 43v135q0 29 21 47q17 17 43 17h5q160 -13 306 -80.5 t259 -181.5q114 -113 181.5 -259t80.5 -306zM1408 67q2 -27 -18 -47q-18 -20 -46 -20h-143q-26 0 -44.5 17.5t-19.5 42.5q-12 215 -101 408.5t-231.5 336t-336 231.5t-408.5 102q-25 1 -42.5 19.5t-17.5 43.5v143q0 28 20 46q18 18 44 18h3q262 -13 501.5 -120t425.5 -294 q187 -186 294 -425.5t120 -501.5z" />
-<glyph unicode="&#xf0a0;" d="M1040 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1296 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1408 160v320q0 13 -9.5 22.5t-22.5 9.5 h-1216q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5zM178 640h1180l-157 482q-4 13 -16 21.5t-26 8.5h-782q-14 0 -26 -8.5t-16 -21.5zM1536 480v-320q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v320q0 25 16 75 l197 606q17 53 63 86t101 33h782q55 0 101 -33t63 -86l197 -606q16 -50 16 -75z" />
-<glyph unicode="&#xf0a1;" horiz-adv-x="1792" d="M1664 896q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5v-384q0 -52 -38 -90t-90 -38q-417 347 -812 380q-58 -19 -91 -66t-31 -100.5t40 -92.5q-20 -33 -23 -65.5t6 -58t33.5 -55t48 -50t61.5 -50.5q-29 -58 -111.5 -83t-168.5 -11.5t-132 55.5q-7 23 -29.5 87.5 t-32 94.5t-23 89t-15 101t3.5 98.5t22 110.5h-122q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h480q435 0 896 384q52 0 90 -38t38 -90v-384zM1536 292v954q-394 -302 -768 -343v-270q377 -42 768 -341z" />
-<glyph unicode="&#xf0a2;" horiz-adv-x="1664" d="M848 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM183 128h1298q-164 181 -246.5 411.5t-82.5 484.5q0 256 -320 256t-320 -256q0 -254 -82.5 -484.5t-246.5 -411.5zM1664 128q0 -52 -38 -90t-90 -38 h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q190 161 287 397.5t97 498.5q0 165 96 262t264 117q-8 18 -8 37q0 40 28 68t68 28t68 -28t28 -68q0 -19 -8 -37q168 -20 264 -117t96 -262q0 -262 97 -498.5t287 -397.5z" />
-<glyph unicode="&#xf0a3;" d="M1376 640l138 -135q30 -28 20 -70q-12 -41 -52 -51l-188 -48l53 -186q12 -41 -19 -70q-29 -31 -70 -19l-186 53l-48 -188q-10 -40 -51 -52q-12 -2 -19 -2q-31 0 -51 22l-135 138l-135 -138q-28 -30 -70 -20q-41 11 -51 52l-48 188l-186 -53q-41 -12 -70 19q-31 29 -19 70 l53 186l-188 48q-40 10 -52 51q-10 42 20 70l138 135l-138 135q-30 28 -20 70q12 41 52 51l188 48l-53 186q-12 41 19 70q29 31 70 19l186 -53l48 188q10 41 51 51q41 12 70 -19l135 -139l135 139q29 30 70 19q41 -10 51 -51l48 -188l186 53q41 12 70 -19q31 -29 19 -70 l-53 -186l188 -48q40 -10 52 -51q10 -42 -20 -70z" />
-<glyph unicode="&#xf0a4;" horiz-adv-x="1792" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 768q0 51 -39 89.5t-89 38.5h-576q0 20 15 48.5t33 55t33 68t15 84.5q0 67 -44.5 97.5t-115.5 30.5q-24 0 -90 -139q-24 -44 -37 -65q-40 -64 -112 -145q-71 -81 -101 -106 q-69 -57 -140 -57h-32v-640h32q72 0 167 -32t193.5 -64t179.5 -32q189 0 189 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5h331q52 0 90 38t38 90zM1792 769q0 -105 -75.5 -181t-180.5 -76h-169q-4 -62 -37 -119q3 -21 3 -43 q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5q-133 0 -322 69q-164 59 -223 59h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h288q10 0 21.5 4.5t23.5 14t22.5 18t24 22.5t20.5 21.5t19 21.5t14 17q65 74 100 129q13 21 33 62t37 72t40.5 63t55 49.5 t69.5 17.5q125 0 206.5 -67t81.5 -189q0 -68 -22 -128h374q104 0 180 -76t76 -179z" />
-<glyph unicode="&#xf0a5;" horiz-adv-x="1792" d="M1376 128h32v640h-32q-35 0 -67.5 12t-62.5 37t-50 46t-49 54q-2 3 -3.5 4.5t-4 4.5t-4.5 5q-72 81 -112 145q-14 22 -38 68q-1 3 -10.5 22.5t-18.5 36t-20 35.5t-21.5 30.5t-18.5 11.5q-71 0 -115.5 -30.5t-44.5 -97.5q0 -43 15 -84.5t33 -68t33 -55t15 -48.5h-576 q-50 0 -89 -38.5t-39 -89.5q0 -52 38 -90t90 -38h331q-15 -17 -25 -47.5t-10 -55.5q0 -69 53 -119q-18 -32 -18 -69t17.5 -73.5t47.5 -52.5q-4 -24 -4 -56q0 -85 48.5 -126t135.5 -41q84 0 183 32t194 64t167 32zM1664 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45 t45 -19t45 19t19 45zM1792 768v-640q0 -53 -37.5 -90.5t-90.5 -37.5h-288q-59 0 -223 -59q-190 -69 -317 -69q-142 0 -230 77.5t-87 217.5l1 5q-61 76 -61 178q0 22 3 43q-33 57 -37 119h-169q-105 0 -180.5 76t-75.5 181q0 103 76 179t180 76h374q-22 60 -22 128 q0 122 81.5 189t206.5 67q38 0 69.5 -17.5t55 -49.5t40.5 -63t37 -72t33 -62q35 -55 100 -129q2 -3 14 -17t19 -21.5t20.5 -21.5t24 -22.5t22.5 -18t23.5 -14t21.5 -4.5h288q53 0 90.5 -37.5t37.5 -90.5z" />
-<glyph unicode="&#xf0a6;" d="M1280 -64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 700q0 189 -167 189q-26 0 -56 -5q-16 30 -52.5 47.5t-73.5 17.5t-69 -18q-50 53 -119 53q-25 0 -55.5 -10t-47.5 -25v331q0 52 -38 90t-90 38q-51 0 -89.5 -39t-38.5 -89v-576 q-20 0 -48.5 15t-55 33t-68 33t-84.5 15q-67 0 -97.5 -44.5t-30.5 -115.5q0 -24 139 -90q44 -24 65 -37q64 -40 145 -112q81 -71 106 -101q57 -69 57 -140v-32h640v32q0 72 32 167t64 193.5t32 179.5zM1536 705q0 -133 -69 -322q-59 -164 -59 -223v-288q0 -53 -37.5 -90.5 t-90.5 -37.5h-640q-53 0 -90.5 37.5t-37.5 90.5v288q0 10 -4.5 21.5t-14 23.5t-18 22.5t-22.5 24t-21.5 20.5t-21.5 19t-17 14q-74 65 -129 100q-21 13 -62 33t-72 37t-63 40.5t-49.5 55t-17.5 69.5q0 125 67 206.5t189 81.5q68 0 128 -22v374q0 104 76 180t179 76 q105 0 181 -75.5t76 -180.5v-169q62 -4 119 -37q21 3 43 3q101 0 178 -60q139 1 219.5 -85t80.5 -227z" />
-<glyph unicode="&#xf0a7;" d="M1408 576q0 84 -32 183t-64 194t-32 167v32h-640v-32q0 -35 -12 -67.5t-37 -62.5t-46 -50t-54 -49q-9 -8 -14 -12q-81 -72 -145 -112q-22 -14 -68 -38q-3 -1 -22.5 -10.5t-36 -18.5t-35.5 -20t-30.5 -21.5t-11.5 -18.5q0 -71 30.5 -115.5t97.5 -44.5q43 0 84.5 15t68 33 t55 33t48.5 15v-576q0 -50 38.5 -89t89.5 -39q52 0 90 38t38 90v331q46 -35 103 -35q69 0 119 53q32 -18 69 -18t73.5 17.5t52.5 47.5q24 -4 56 -4q85 0 126 48.5t41 135.5zM1280 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 580 q0 -142 -77.5 -230t-217.5 -87l-5 1q-76 -61 -178 -61q-22 0 -43 3q-54 -30 -119 -37v-169q0 -105 -76 -180.5t-181 -75.5q-103 0 -179 76t-76 180v374q-54 -22 -128 -22q-121 0 -188.5 81.5t-67.5 206.5q0 38 17.5 69.5t49.5 55t63 40.5t72 37t62 33q55 35 129 100 q3 2 17 14t21.5 19t21.5 20.5t22.5 24t18 22.5t14 23.5t4.5 21.5v288q0 53 37.5 90.5t90.5 37.5h640q53 0 90.5 -37.5t37.5 -90.5v-288q0 -59 59 -223q69 -190 69 -317z" />
-<glyph unicode="&#xf0a8;" d="M1280 576v128q0 26 -19 45t-45 19h-502l189 189q19 19 19 45t-19 45l-91 91q-18 18 -45 18t-45 -18l-362 -362l-91 -91q-18 -18 -18 -45t18 -45l91 -91l362 -362q18 -18 45 -18t45 18l91 91q18 18 18 45t-18 45l-189 189h502q26 0 45 19t19 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0a9;" d="M1285 640q0 27 -18 45l-91 91l-362 362q-18 18 -45 18t-45 -18l-91 -91q-18 -18 -18 -45t18 -45l189 -189h-502q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h502l-189 -189q-19 -19 -19 -45t19 -45l91 -91q18 -18 45 -18t45 18l362 362l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0aa;" d="M1284 641q0 27 -18 45l-362 362l-91 91q-18 18 -45 18t-45 -18l-91 -91l-362 -362q-18 -18 -18 -45t18 -45l91 -91q18 -18 45 -18t45 18l189 189v-502q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v502l189 -189q19 -19 45 -19t45 19l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0ab;" d="M1284 639q0 27 -18 45l-91 91q-18 18 -45 18t-45 -18l-189 -189v502q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-502l-189 189q-19 19 -45 19t-45 -19l-91 -91q-18 -18 -18 -45t18 -45l362 -362l91 -91q18 -18 45 -18t45 18l91 91l362 362q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0ac;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1042 887q-2 -1 -9.5 -9.5t-13.5 -9.5q2 0 4.5 5t5 11t3.5 7q6 7 22 15q14 6 52 12q34 8 51 -11 q-2 2 9.5 13t14.5 12q3 2 15 4.5t15 7.5l2 22q-12 -1 -17.5 7t-6.5 21q0 -2 -6 -8q0 7 -4.5 8t-11.5 -1t-9 -1q-10 3 -15 7.5t-8 16.5t-4 15q-2 5 -9.5 10.5t-9.5 10.5q-1 2 -2.5 5.5t-3 6.5t-4 5.5t-5.5 2.5t-7 -5t-7.5 -10t-4.5 -5q-3 2 -6 1.5t-4.5 -1t-4.5 -3t-5 -3.5 q-3 -2 -8.5 -3t-8.5 -2q15 5 -1 11q-10 4 -16 3q9 4 7.5 12t-8.5 14h5q-1 4 -8.5 8.5t-17.5 8.5t-13 6q-8 5 -34 9.5t-33 0.5q-5 -6 -4.5 -10.5t4 -14t3.5 -12.5q1 -6 -5.5 -13t-6.5 -12q0 -7 14 -15.5t10 -21.5q-3 -8 -16 -16t-16 -12q-5 -8 -1.5 -18.5t10.5 -16.5 q2 -2 1.5 -4t-3.5 -4.5t-5.5 -4t-6.5 -3.5l-3 -2q-11 -5 -20.5 6t-13.5 26q-7 25 -16 30q-23 8 -29 -1q-5 13 -41 26q-25 9 -58 4q6 1 0 15q-7 15 -19 12q3 6 4 17.5t1 13.5q3 13 12 23q1 1 7 8.5t9.5 13.5t0.5 6q35 -4 50 11q5 5 11.5 17
 t10.5 17q9 6 14 5.5t14.5 -5.5 t14.5 -5q14 -1 15.5 11t-7.5 20q12 -1 3 17q-5 7 -8 9q-12 4 -27 -5q-8 -4 2 -8q-1 1 -9.5 -10.5t-16.5 -17.5t-16 5q-1 1 -5.5 13.5t-9.5 13.5q-8 0 -16 -15q3 8 -11 15t-24 8q19 12 -8 27q-7 4 -20.5 5t-19.5 -4q-5 -7 -5.5 -11.5t5 -8t10.5 -5.5t11.5 -4t8.5 -3 q14 -10 8 -14q-2 -1 -8.5 -3.5t-11.5 -4.5t-6 -4q-3 -4 0 -14t-2 -14q-5 5 -9 17.5t-7 16.5q7 -9 -25 -6l-10 1q-4 0 -16 -2t-20.5 -1t-13.5 8q-4 8 0 20q1 4 4 2q-4 3 -11 9.5t-10 8.5q-46 -15 -94 -41q6 -1 12 1q5 2 13 6.5t10 5.5q34 14 42 7l5 5q14 -16 20 -25 q-7 4 -30 1q-20 -6 -22 -12q7 -12 5 -18q-4 3 -11.5 10t-14.5 11t-15 5q-16 0 -22 -1q-146 -80 -235 -222q7 -7 12 -8q4 -1 5 -9t2.5 -11t11.5 3q9 -8 3 -19q1 1 44 -27q19 -17 21 -21q3 -11 -10 -18q-1 2 -9 9t-9 4q-3 -5 0.5 -18.5t10.5 -12.5q-7 0 -9.5 -16t-2.5 -35.5 t-1 -23.5l2 -1q-3 -12 5.5 -34.5t21.5 -19.5q-13 -3 20 -43q6 -8 8 -9q3 -2 12 -7.5t15 -10t10 -10.5q4 -5 10 -22.5t14 -23.5q-2 -6 9.5 -20t10.5 -23q-1 0 -2.5 -1t-2.5 -1q3 -7 15.5 -14t15.5 -13q1 -3 2 -10t3 -11t8 -2q2 20 -24 62q-1
 5 25 -17 29q-3 5 -5.5 15.5 t-4.5 14.5q2 0 6 -1.5t8.5 -3.5t7.5 -4t2 -3q-3 -7 2 -17.5t12 -18.5t17 -19t12 -13q6 -6 14 -19.5t0 -13.5q9 0 20 -10t17 -20q5 -8 8 -26t5 -24q2 -7 8.5 -13.5t12.5 -9.5l16 -8t13 -7q5 -2 18.5 -10.5t21.5 -11.5q10 -4 16 -4t14.5 2.5t13.5 3.5q15 2 29 -15t21 -21 q36 -19 55 -11q-2 -1 0.5 -7.5t8 -15.5t9 -14.5t5.5 -8.5q5 -6 18 -15t18 -15q6 4 7 9q-3 -8 7 -20t18 -10q14 3 14 32q-31 -15 -49 18q0 1 -2.5 5.5t-4 8.5t-2.5 8.5t0 7.5t5 3q9 0 10 3.5t-2 12.5t-4 13q-1 8 -11 20t-12 15q-5 -9 -16 -8t-16 9q0 -1 -1.5 -5.5t-1.5 -6.5 q-13 0 -15 1q1 3 2.5 17.5t3.5 22.5q1 4 5.5 12t7.5 14.5t4 12.5t-4.5 9.5t-17.5 2.5q-19 -1 -26 -20q-1 -3 -3 -10.5t-5 -11.5t-9 -7q-7 -3 -24 -2t-24 5q-13 8 -22.5 29t-9.5 37q0 10 2.5 26.5t3 25t-5.5 24.5q3 2 9 9.5t10 10.5q2 1 4.5 1.5t4.5 0t4 1.5t3 6q-1 1 -4 3 q-3 3 -4 3q7 -3 28.5 1.5t27.5 -1.5q15 -11 22 2q0 1 -2.5 9.5t-0.5 13.5q5 -27 29 -9q3 -3 15.5 -5t17.5 -5q3 -2 7 -5.5t5.5 -4.5t5 0.5t8.5 6.5q10 -14 12 -24q11 -40 19 -44q7 -3 11 -2t4.5 9.5t0 14t-1.5 12.5l-1 8v18l-1 8q
 -15 3 -18.5 12t1.5 18.5t15 18.5q1 1 8 3.5 t15.5 6.5t12.5 8q21 19 15 35q7 0 11 9q-1 0 -5 3t-7.5 5t-4.5 2q9 5 2 16q5 3 7.5 11t7.5 10q9 -12 21 -2q7 8 1 16q5 7 20.5 10.5t18.5 9.5q7 -2 8 2t1 12t3 12q4 5 15 9t13 5l17 11q3 4 0 4q18 -2 31 11q10 11 -6 20q3 6 -3 9.5t-15 5.5q3 1 11.5 0.5t10.5 1.5 q15 10 -7 16q-17 5 -43 -12zM879 10q206 36 351 189q-3 3 -12.5 4.5t-12.5 3.5q-18 7 -24 8q1 7 -2.5 13t-8 9t-12.5 8t-11 7q-2 2 -7 6t-7 5.5t-7.5 4.5t-8.5 2t-10 -1l-3 -1q-3 -1 -5.5 -2.5t-5.5 -3t-4 -3t0 -2.5q-21 17 -36 22q-5 1 -11 5.5t-10.5 7t-10 1.5t-11.5 -7 q-5 -5 -6 -15t-2 -13q-7 5 0 17.5t2 18.5q-3 6 -10.5 4.5t-12 -4.5t-11.5 -8.5t-9 -6.5t-8.5 -5.5t-8.5 -7.5q-3 -4 -6 -12t-5 -11q-2 4 -11.5 6.5t-9.5 5.5q2 -10 4 -35t5 -38q7 -31 -12 -48q-27 -25 -29 -40q-4 -22 12 -26q0 -7 -8 -20.5t-7 -21.5q0 -6 2 -16z" />
-<glyph unicode="&#xf0ad;" horiz-adv-x="16

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/font/fontawesome-webfont-ttf.ttf
----------------------------------------------------------------------
diff --git a/gemfire-site/content/font/fontawesome-webfont-ttf.ttf b/gemfire-site/content/font/fontawesome-webfont-ttf.ttf
deleted file mode 100755
index d365924..0000000
Binary files a/gemfire-site/content/font/fontawesome-webfont-ttf.ttf and /dev/null differ



[33/50] [abbrv] incubator-geode git commit: GEODE-608: Exclude local gradle dirs

Posted by je...@apache.org.
GEODE-608:  Exclude local gradle dirs

When Jenkins runs a gradle build, it sets the GRADLE_USER_HOME env
to the working dir.  This creates caches/ daemon/ native/ paths
that cause excessive RAT runtime and errors.  This change exclude
those from the RAT checks.


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

Branch: refs/heads/feature/GEODE-14
Commit: 0b288a21d504d08803ecf053f74228bda4211972
Parents: 15b0655
Author: Anthony Baker <ab...@pivotal.io>
Authored: Tue Dec 15 09:45:41 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Tue Dec 15 10:06:02 2015 -0800

----------------------------------------------------------------------
 gradle/rat.gradle | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0b288a21/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 79fe2bb..63eff0b 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -27,6 +27,10 @@ rat {
     'gradlew',
     'gradlew.bat',
     'gradle/wrapper/gradle-wrapper.properties',
+    'caches/**',
+    'daemon/**',
+    'native/**',
+    'wrapper/**',
     '**/build/**',
 
     // IDE


[46/50] [abbrv] incubator-geode git commit: Revision bb794ee205400ac4e5655cc1e6410889d29f28b1 closed #61

Posted by je...@apache.org.
Revision bb794ee205400ac4e5655cc1e6410889d29f28b1 closed #61


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

Branch: refs/heads/feature/GEODE-14
Commit: 5d1f38ed3c2bcff5871630abbe92df032988b881
Parents: bb794ee
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Dec 17 14:27:48 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 17 14:27:48 2015 -0800

----------------------------------------------------------------------

----------------------------------------------------------------------



[47/50] [abbrv] incubator-geode git commit: implementation of AvailablePort.isPortAvailable for multicast

Posted by je...@apache.org.
implementation of AvailablePort.isPortAvailable for multicast

The AvailablePort utility has not had support for finding a free multicast
port since feature/GEODE-77 was merged to the develop branch.  This change-
set enables the old multicast code in AvailablePort and implements the
check & response for its datagrams in JGroupsMessenger.  The JGroups
transport method for receiving packets is overridden to look for these
datagrams and respond to them.

I also renamed AvailablePort.JGroups to AvailablePort.MULTICAST, which
describes the kind of port that's being tested a little better.


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

Branch: refs/heads/feature/GEODE-14
Commit: b826b4b0077dc159cbcee68775cc197b025d5586
Parents: 5d1f38e
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Dec 18 08:31:52 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Dec 18 08:31:52 2015 -0800

----------------------------------------------------------------------
 .../membership/gms/messenger/Transport.java     |  26 +++++
 .../gemfire/internal/AvailablePort.java         | 103 +++++++++----------
 .../distributed/DistributedSystemDUnitTest.java |   6 +-
 .../gemfire/distributed/LocatorDUnitTest.java   |   2 +-
 .../messenger/JGroupsMessengerJUnitTest.java    |   4 +-
 .../gemfire/internal/AvailablePortHelper.java   |   2 +-
 .../internal/AvailablePortJUnitTest.java        |   2 +
 7 files changed, 86 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
index 1687261..fbb4b28 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/Transport.java
@@ -138,5 +138,31 @@ public class Transport extends UDP {
     }
   }
 
+  // overridden to implement AvailablePort response
+  @Override
+  public void receive(Address sender, byte[] data, int offset, int length, boolean copy_buffer) {
+    if(data == null) return;
+
+    // drop message from self; it has already been looped back up (https://issues.jboss.org/browse/JGRP-1765)
+    if(local_physical_addr != null && local_physical_addr.equals(sender))
+        return;
+
+    if (length-offset == 4
+        && data[offset] == 'p'
+        && data[offset+1] == 'i'
+        && data[offset+2] == 'n'
+        && data[offset+3] == 'g') {
+      // AvailablePort check
+      data[offset+1] = 'o';
+      try {
+        sendToSingleMember(sender, data, offset, length);
+      } catch (Exception e) {
+        log.fatal("Unable to respond to available-port check", e);
+      }
+      return;
+    }
+
+    super.receive(sender,  data,  offset,  length, copy_buffer);
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
index c4907ad..93c6bc2 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/AvailablePort.java
@@ -35,8 +35,8 @@ public class AvailablePort {
   /** Is the port available for a Socket (TCP) connection? */
   public static final int SOCKET = 0;
 
-  /** Is the port available for a JGroups (UDP) connection */
-  public static final int JGROUPS = 1;
+  /** Is the port available for a JGroups (UDP) multicast connection */
+  public static final int MULTICAST = 1;
 
   ///////////////////////  Static Methods  ///////////////////////
   
@@ -50,7 +50,7 @@ public class AvailablePort {
       if (protocol == SOCKET) {
         name = System.getProperty("gemfire.bind-address");
       }
-      else if (protocol == JGROUPS) {
+      else if (protocol == MULTICAST) {
         name = System.getProperty("gemfire.mcast-address");
       }
       if (name != null) {
@@ -72,7 +72,7 @@ public class AvailablePort {
    *        The port to check
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    *
    * @throws IllegalArgumentException
    *         <code>protocol</code> is unknown
@@ -90,7 +90,7 @@ public class AvailablePort {
    *        The port to check
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    * @param addr the bind address (or mcast address) to use
    *
    * @throws IllegalArgumentException
@@ -106,47 +106,44 @@ public class AvailablePort {
       }
     }
     
-    else if (protocol == JGROUPS) {
+    else if (protocol == MULTICAST) {
       DatagramSocket socket = null;
       try {
-        // TODO - need to find out if anyone is listening on this port
-        return true;
-
-//        socket = new MulticastSocket();
-//        socket.setSoTimeout(Integer.getInteger("AvailablePort.timeout", 2000).intValue());
-//        byte[] buffer = new byte[4];
-//        buffer[0] = (byte)'p';
-//        buffer[1] = (byte)'i';
-//        buffer[2] = (byte)'n';
-//        buffer[3] = (byte)'g';
-//        SocketAddress mcaddr = new InetSocketAddress(
-//          addr==null? DistributionConfig.DEFAULT_MCAST_ADDRESS : addr, port);
-//        DatagramPacket packet = new DatagramPacket(buffer, 0, buffer.length, mcaddr);
-//        socket.send(packet);
-//        try {
-//          socket.receive(packet);
-//          packet.getData();  // make sure there's data, but no need to process it
-//          return false;
-//        }
-//        catch (SocketTimeoutException ste) {
-//          //System.out.println("socket read timed out");
-//          return true;
-//        }
-//        catch (Exception e) {
-//          e.printStackTrace();
-//          return false;
-//        }
-//      }
-//      catch (java.io.IOException ioe) {
-//        if (ioe.getMessage().equals("Network is unreachable")) {
-//          throw new RuntimeException(LocalizedStrings.AvailablePort_NETWORK_IS_UNREACHABLE.toLocalizedString(), ioe);
-//        }
-//        ioe.printStackTrace();
-//        return false;
-//      }
-//      catch (Exception e) {
-//        e.printStackTrace();
-//        return false;
+        socket = new MulticastSocket();
+        socket.setSoTimeout(Integer.getInteger("AvailablePort.timeout", 2000).intValue());
+        byte[] buffer = new byte[4];
+        buffer[0] = (byte)'p';
+        buffer[1] = (byte)'i';
+        buffer[2] = (byte)'n';
+        buffer[3] = (byte)'g';
+        SocketAddress mcaddr = new InetSocketAddress(
+          addr==null? DistributionConfig.DEFAULT_MCAST_ADDRESS : addr, port);
+        DatagramPacket packet = new DatagramPacket(buffer, 0, buffer.length, mcaddr);
+        socket.send(packet);
+        try {
+          socket.receive(packet);
+          packet.getData();  // make sure there's data, but no need to process it
+          return false;
+        }
+        catch (SocketTimeoutException ste) {
+          //System.out.println("socket read timed out");
+          return true;
+        }
+        catch (Exception e) {
+          e.printStackTrace();
+          return false;
+        }
+      }
+      catch (java.io.IOException ioe) {
+        if (ioe.getMessage().equals("Network is unreachable")) {
+          throw new RuntimeException(LocalizedStrings.AvailablePort_NETWORK_IS_UNREACHABLE.toLocalizedString(), ioe);
+        }
+        ioe.printStackTrace();
+        return false;
+      }
+      catch (Exception e) {
+        e.printStackTrace();
+        return false;
       }
       finally {
         if (socket != null) {
@@ -173,7 +170,7 @@ public class AvailablePort {
       } else {
         return keepOneInterface(addr, port);
       }
-    } else if (protocol == JGROUPS) {
+    } else if (protocol == MULTICAST) {
       throw new IllegalArgumentException("You can not keep the JGROUPS protocol");
     } else {
       throw new IllegalArgumentException(LocalizedStrings.AvailablePort_UNKNOWN_PROTOCOL_0.toLocalizedString(Integer.valueOf(protocol)));
@@ -297,7 +294,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    *
    * @throws IllegalArgumentException
    *         <code>protocol</code> is unknown
@@ -314,7 +311,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    *
    * @throws IllegalArgumentException
    *         <code>protocol</code> is unknown
@@ -329,7 +326,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    *
    * @throws IllegalArgumentException
    *         <code>protocol</code> is unknown
@@ -345,7 +342,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    * @param addr the bind-address or mcast address to use
    *
    * @throws IllegalArgumentException
@@ -356,7 +353,7 @@ public class AvailablePort {
       int port = getRandomWildcardBindPortNumber();
       if (isPortAvailable(port, protocol, addr)) {
         // don't return the products default multicast port
-        if ( !(protocol == JGROUPS && port == DistributionConfig.DEFAULT_MCAST_PORT) ){
+        if ( !(protocol == MULTICAST && port == DistributionConfig.DEFAULT_MCAST_PORT) ){
           return port;
         }
       }
@@ -377,7 +374,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    * @param addr the bind-address or mcast address to use
    *
    * @throws IllegalArgumentException
@@ -399,7 +396,7 @@ public class AvailablePort {
    *
    * @param protocol
    *        The protocol to check (either {@link #SOCKET} or {@link
-   *        #JGROUPS}). 
+   *        #MULTICAST}). 
    * @param addr the bind-address or mcast address to use
    *
    * @throws IllegalArgumentException
@@ -543,7 +540,7 @@ public class AvailablePort {
 
     } else if (protocolString.equalsIgnoreCase("javagroups") ||
       protocolString.equalsIgnoreCase("jgroups")) {
-      protocol = JGROUPS;
+      protocol = MULTICAST;
 
     } else {
       usage("Unknown protocol: " + protocolString);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
index a3ff8df..9f1e90b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemDUnitTest.java
@@ -317,7 +317,7 @@ public class DistributedSystemDUnitTest extends DistributedTestCase {
   // TODO this needs to use a locator
   public void _testMembershipPortRange() throws Exception {
     Properties config = new Properties();
-    int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     int unicastPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     config.setProperty("mcast-port", String.valueOf(mcastPort));
     config.setProperty("locators", "");
@@ -337,7 +337,7 @@ public class DistributedSystemDUnitTest extends DistributedTestCase {
   // TODO this needs to use a locator
   public void _testMembershipPortRangeWithExactThreeValues() throws Exception {
     Properties config = new Properties();
-    int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     config.setProperty("mcast-port", String.valueOf(mcastPort));
     config.setProperty("locators", "");
     config.setProperty(DistributionConfig.MEMBERSHIP_PORT_RANGE_NAME, ""
@@ -358,7 +358,7 @@ public class DistributedSystemDUnitTest extends DistributedTestCase {
   // TODO this needs to use a locator
   public void _testConflictingUDPPort() throws Exception {
     final Properties config = new Properties();
-    final int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    final int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     final int unicastPort = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
     config.setProperty("mcast-port", String.valueOf(mcastPort));
     config.setProperty("locators", "");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
index 7d34ab1..80f527b 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
@@ -1454,7 +1454,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
     final int port2 = freeTCPPorts[1];
     this.port2 = port2;
     deleteLocatorStateFile(port1, port2);
-    final int mcastport = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+    final int mcastport = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
     
     final String host0 = getServerHostName(host); 
     final String locators = host0 + "[" + port1 + "]," +

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
index c844583..9d3dbac 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
@@ -68,6 +68,7 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinReq
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.JoinResponseMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messages.LeaveRequestMessage;
 import com.gemstone.gemfire.distributed.internal.membership.gms.messenger.JGroupsMessenger.JGroupsReceiver;
+import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
@@ -812,8 +813,9 @@ public class JGroupsMessengerJUnitTest {
   public void testMulticastTest() throws Exception {
     initMocks(true);
     boolean result = messenger.testMulticast(50);
-    // this shouldln't succeed
+    // this shouldln't succeed because there's no-one to respond
     assertFalse(result);
+    assertFalse(AvailablePort.isPortAvailable(services.getConfig().getDistributionConfig().getMcastPort(), AvailablePort.MULTICAST));
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java
index 5230ff9..246e6c7 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortHelper.java
@@ -110,7 +110,7 @@ public class AvailablePortHelper {
   public static int[] getRandomAvailableUDPPorts(int count) {
     Set set = new HashSet();
     while (set.size() < count) {
-      int port = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
+      int port = AvailablePort.getRandomAvailablePort(AvailablePort.MULTICAST);
       set.add(new Integer(port));
     }
     int[] ports = new int[set.size()];

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b826b4b0/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
index 7244822..6ba91b8 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/AvailablePortJUnitTest.java
@@ -69,4 +69,6 @@ public class AvailablePortJUnitTest {
     socket.bind(new InetSocketAddress((InetAddress)null, port));
     assertFalse(AvailablePort.isPortAvailable(port, AvailablePort.SOCKET));
   }
+  
+  // multicast availability is tested in JGroupsMessengerJUnitTest
 }


[38/50] [abbrv] incubator-geode git commit: GEODE-668 Improve GMSHealthMonitor logs to help debugging

Posted by je...@apache.org.
GEODE-668 Improve GMSHealthMonitor logs to help debugging


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

Branch: refs/heads/feature/GEODE-14
Commit: e63ad64b693c10f3de2809697a1b60eef8514e38
Parents: df32ad1
Author: Jianxia Chen <jc...@pivotal.io>
Authored: Tue Dec 15 14:51:31 2015 -0800
Committer: Jianxia Chen <jc...@pivotal.io>
Committed: Tue Dec 15 14:51:31 2015 -0800

----------------------------------------------------------------------
 .../internal/membership/gms/fd/GMSHealthMonitor.java      | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e63ad64b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
index 8c38ba6..2aac935 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
@@ -514,7 +514,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
         GMSMember gmbr = (GMSMember) suspectMember.getNetMember();
         writeMemberToStream(gmbr, out);
         clientSocket.shutdownOutput();
-        logger.debug("Connected - reading response", suspectMember);
+        logger.debug("Connected - reading response from suspect member {}", suspectMember);
         int b = in.read();
         logger.debug("Received {}", (b == OK ? "OK" : (b == ERROR ? "ERROR" : b)), suspectMember);
         if (b == OK) {
@@ -531,7 +531,7 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
         return false;
       }
     } catch (SocketTimeoutException e) {
-      logger.debug("tcp/ip connection timed out");
+      logger.debug("Final check TCP/IP connection timed out for suspect member {}", suspectMember);
       return false;
     } catch (IOException e) {
       logger.trace("Unexpected exception", e);
@@ -1201,15 +1201,15 @@ public class GMSHealthMonitor implements HealthMonitor, MessageHandler {
               if (!pinged && !isStopping) {
                 TimeStamp ts = memberTimeStamps.get(mbr);
                 if (ts == null || ts.getTime() <= startTime) {
-                  logger.info("Final check failed - requesting removal");
+                  logger.info("Final check failed - requesting removal of suspect member " + mbr);
                   services.getJoinLeave().remove(mbr, reason);
                   failed = true;
                 } else {
-                  logger.info("check failed but detected recent message traffic");
+                  logger.info("Final check failed but detected recent message traffic for suspect member " + mbr);
                 }
               }
               if (!failed) {
-                logger.info("Final check passed");
+                logger.info("Final check passed for suspect member " + mbr);
               }
               // whether it's alive or not, at this point we allow it to
               // be watched again


[49/50] [abbrv] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-14

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


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

Branch: refs/heads/feature/GEODE-14
Commit: cf95428774af3e567e1923463078cee990f00fc5
Parents: 4b56f5e 0851f18
Author: Jens Deppe <jd...@pivotal.io>
Authored: Fri Dec 18 16:06:08 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri Dec 18 16:06:08 2015 -0800

----------------------------------------------------------------------
 README.md                                       |   11 +-
 build.gradle                                    |  142 +-
 dev-tools/docker/base/Dockerfile                |   54 +
 dev-tools/docker/base/build-base-docker.sh      |   30 +
 dev-tools/docker/compile/Dockerfile             |   21 +
 .../docker/compile/start-compile-docker.sh      |   62 +
 docker/Dockerfile                               |   46 +
 docker/README.md                                |   67 +
 docker/build-runtime-docker.sh                  |   32 +
 etc/eclipseFormatterProfile.xml                 |   11 +-
 gemfire-assembly/build.gradle                   |   88 +-
 gemfire-assembly/src/main/dist/bin/gfsh         |   16 +-
 .../src/main/dist/bin/gfsh-completion.bash      |   15 +
 gemfire-assembly/src/main/dist/bin/gfsh.bat     |   16 +-
 .../src/test/java/AgentUtilJUnitTest.java       |  109 -
 .../LocatorLauncherAssemblyJUnitTest.java       |  157 +
 .../management/internal/AgentUtilJUnitTest.java |  106 +
 .../LauncherLifecycleCommandsDUnitTest.java     | 1005 ++++
 .../LauncherLifecycleCommandsJUnitTest.java     |  625 +++
 .../SharedConfigurationEndToEndDUnitTest.java   |  443 ++
 gemfire-common/build.gradle                     |   20 +
 .../gemfire/annotations/Experimental.java       |   56 +
 .../annotations/ExperimentalJUnitTest.java      |  199 +
 .../ClassInExperimentalPackage.java             |   27 +
 .../experimentalpackage/package-info.java       |   27 +
 .../ClassInNonExperimentalPackage.java          |   27 +
 .../nonexperimentalpackage/package-info.java    |   24 +
 gemfire-core/build.gradle                       |  204 +-
 .../internal/ra/GFConnectionFactoryImpl.java    |   21 +-
 .../gemfire/internal/ra/GFConnectionImpl.java   |   21 +-
 .../internal/ra/spi/JCALocalTransaction.java    |   21 +-
 .../internal/ra/spi/JCAManagedConnection.java   |   21 +-
 .../ra/spi/JCAManagedConnectionFactory.java     |   21 +-
 .../ra/spi/JCAManagedConnectionMetaData.java    |   21 +-
 gemfire-core/src/jca/ra.xml                     |   19 +-
 .../com/gemstone/gemfire/CancelCriterion.java   |   21 +-
 .../com/gemstone/gemfire/CancelException.java   |   21 +-
 .../gemstone/gemfire/CanonicalInstantiator.java |   21 +-
 .../com/gemstone/gemfire/CopyException.java     |   21 +-
 .../java/com/gemstone/gemfire/CopyHelper.java   |   21 +-
 .../com/gemstone/gemfire/DataSerializable.java  |   21 +-
 .../com/gemstone/gemfire/DataSerializer.java    |   34 +-
 .../main/java/com/gemstone/gemfire/Delta.java   |   21 +-
 .../gemfire/DeltaSerializationException.java    |   21 +-
 .../gemfire/ForcedDisconnectException.java      |   21 +-
 .../gemstone/gemfire/GemFireCacheException.java |   21 +-
 .../gemfire/GemFireCheckedException.java        |   21 +-
 .../gemfire/GemFireConfigException.java         |   21 +-
 .../com/gemstone/gemfire/GemFireException.java  |   21 +-
 .../gemstone/gemfire/GemFireIOException.java    |   21 +-
 .../gemstone/gemfire/GemFireRethrowable.java    |   21 +-
 .../gemfire/IncompatibleSystemException.java    |   21 +-
 .../java/com/gemstone/gemfire/Instantiator.java |   21 +-
 .../gemstone/gemfire/InternalGemFireError.java  |   21 +-
 .../gemfire/InternalGemFireException.java       |   21 +-
 .../gemstone/gemfire/InvalidDeltaException.java |   21 +-
 .../gemstone/gemfire/InvalidValueException.java |   21 +-
 .../gemfire/InvalidVersionException.java        |   16 +
 .../com/gemstone/gemfire/LicenseException.java  |   21 +-
 .../java/com/gemstone/gemfire/LogWriter.java    |   21 +-
 .../com/gemstone/gemfire/NoSystemException.java |   21 +-
 .../gemfire/OutOfOffHeapMemoryException.java    |   21 +-
 .../gemfire/SerializationException.java         |   21 +-
 .../gemstone/gemfire/StatisticDescriptor.java   |   21 +-
 .../java/com/gemstone/gemfire/Statistics.java   |   21 +-
 .../com/gemstone/gemfire/StatisticsFactory.java |   21 +-
 .../com/gemstone/gemfire/StatisticsType.java    |   21 +-
 .../gemstone/gemfire/StatisticsTypeFactory.java |   21 +-
 .../gemfire/SystemConnectException.java         |   21 +-
 .../com/gemstone/gemfire/SystemFailure.java     |   25 +-
 .../gemfire/SystemIsRunningException.java       |   21 +-
 .../gemfire/ThreadInterruptedException.java     |   18 +-
 .../com/gemstone/gemfire/ToDataException.java   |   21 +-
 .../gemfire/UncreatedSystemException.java       |   21 +-
 .../gemstone/gemfire/UnmodifiableException.java |   21 +-
 .../gemfire/UnstartedSystemException.java       |   21 +-
 .../com/gemstone/gemfire/admin/AdminConfig.java |   21 +-
 .../gemfire/admin/AdminDistributedSystem.java   |   26 +-
 .../admin/AdminDistributedSystemFactory.java    |   21 +-
 .../gemstone/gemfire/admin/AdminException.java  |   21 +-
 .../gemfire/admin/AdminXmlException.java        |   21 +-
 .../java/com/gemstone/gemfire/admin/Alert.java  |   21 +-
 .../com/gemstone/gemfire/admin/AlertLevel.java  |   21 +-
 .../gemstone/gemfire/admin/AlertListener.java   |   21 +-
 .../gemstone/gemfire/admin/BackupStatus.java    |   21 +-
 .../admin/CacheDoesNotExistException.java       |   21 +-
 .../gemfire/admin/CacheHealthConfig.java        |   21 +-
 .../com/gemstone/gemfire/admin/CacheServer.java |   21 +-
 .../gemfire/admin/CacheServerConfig.java        |   21 +-
 .../com/gemstone/gemfire/admin/CacheVm.java     |   21 +-
 .../gemstone/gemfire/admin/CacheVmConfig.java   |   21 +-
 .../gemfire/admin/ConfigurationParameter.java   |   21 +-
 .../gemfire/admin/DistributedSystemConfig.java  |   21 +-
 .../admin/DistributedSystemHealthConfig.java    |   21 +-
 .../gemfire/admin/DistributionLocator.java      |   21 +-
 .../admin/DistributionLocatorConfig.java        |   21 +-
 .../gemstone/gemfire/admin/GemFireHealth.java   |   21 +-
 .../gemfire/admin/GemFireHealthConfig.java      |   21 +-
 .../gemfire/admin/GemFireMemberStatus.java      |   34 +-
 .../gemstone/gemfire/admin/ManagedEntity.java   |   21 +-
 .../gemfire/admin/ManagedEntityConfig.java      |   21 +-
 .../gemfire/admin/MemberHealthConfig.java       |   21 +-
 .../admin/OperationCancelledException.java      |   21 +-
 .../gemfire/admin/RegionNotFoundException.java  |   21 +-
 .../gemfire/admin/RegionSubRegionSnapshot.java  |   21 +-
 .../gemfire/admin/RuntimeAdminException.java    |   21 +-
 .../com/gemstone/gemfire/admin/Statistic.java   |   21 +-
 .../gemfire/admin/StatisticResource.java        |   21 +-
 .../gemstone/gemfire/admin/SystemMember.java    |   21 +-
 .../gemfire/admin/SystemMemberBridgeServer.java |   26 +-
 .../gemfire/admin/SystemMemberCache.java        |   48 +-
 .../gemfire/admin/SystemMemberCacheEvent.java   |   16 +
 .../admin/SystemMemberCacheListener.java        |   21 +-
 .../gemfire/admin/SystemMemberCacheServer.java  |   21 +-
 .../gemfire/admin/SystemMemberRegion.java       |   21 +-
 .../gemfire/admin/SystemMemberRegionEvent.java  |   16 +
 .../gemfire/admin/SystemMemberType.java         |   21 +-
 .../gemfire/admin/SystemMembershipEvent.java    |   21 +-
 .../gemfire/admin/SystemMembershipListener.java |   21 +-
 .../UnmodifiableConfigurationException.java     |   21 +-
 .../admin/internal/AbstractHealthEvaluator.java |   21 +-
 .../internal/AdminDistributedSystemImpl.java    |   45 +-
 .../admin/internal/BackupStatusImpl.java        |   21 +-
 .../admin/internal/CacheHealthConfigImpl.java   |   21 +-
 .../admin/internal/CacheHealthEvaluator.java    |   21 +-
 .../admin/internal/CacheServerConfigImpl.java   |   21 +-
 .../gemfire/admin/internal/CacheServerImpl.java |   21 +-
 .../internal/ConfigurationParameterImpl.java    |   21 +-
 .../ConfigurationParameterListener.java         |   21 +-
 .../DisabledManagedEntityController.java        |   21 +-
 .../internal/DistributedSystemConfigImpl.java   |   22 +-
 .../DistributedSystemHealthConfigImpl.java      |   21 +-
 .../DistributedSystemHealthEvaluator.java       |   23 +-
 .../DistributedSystemHealthMonitor.java         |   23 +-
 .../internal/DistributionLocatorConfigImpl.java |   21 +-
 .../admin/internal/DistributionLocatorImpl.java |   21 +-
 .../EnabledManagedEntityController.java         |   21 +-
 .../admin/internal/FinishBackupRequest.java     |   21 +-
 .../admin/internal/FinishBackupResponse.java    |   21 +-
 .../admin/internal/FlushToDiskRequest.java      |   21 +-
 .../admin/internal/FlushToDiskResponse.java     |   21 +-
 .../admin/internal/GemFireHealthConfigImpl.java |   21 +-
 .../admin/internal/GemFireHealthEvaluator.java  |   21 +-
 .../admin/internal/GemFireHealthImpl.java       |   21 +-
 .../gemfire/admin/internal/InetAddressUtil.java |   23 +-
 .../admin/internal/InternalManagedEntity.java   |   21 +-
 .../gemfire/admin/internal/LogCollator.java     |   21 +-
 .../admin/internal/ManagedEntityConfigImpl.java |   21 +-
 .../admin/internal/ManagedEntityConfigXml.java  |   21 +-
 .../ManagedEntityConfigXmlGenerator.java        |   42 +-
 .../internal/ManagedEntityConfigXmlParser.java  |   21 +-
 .../admin/internal/ManagedEntityController.java |   21 +-
 .../ManagedEntityControllerFactory.java         |   21 +-
 .../admin/internal/ManagedSystemMemberImpl.java |   21 +-
 .../admin/internal/MemberHealthConfigImpl.java  |   21 +-
 .../admin/internal/MemberHealthEvaluator.java   |   21 +-
 .../admin/internal/PrepareBackupRequest.java    |   21 +-
 .../admin/internal/PrepareBackupResponse.java   |   21 +-
 .../gemfire/admin/internal/StatisticImpl.java   |   21 +-
 .../admin/internal/StatisticResourceImpl.java   |   20 +-
 .../internal/SystemMemberBridgeServerImpl.java  |   21 +-
 .../internal/SystemMemberCacheEventImpl.java    |   21 +-
 .../SystemMemberCacheEventProcessor.java        |   21 +-
 .../admin/internal/SystemMemberCacheImpl.java   |   38 +-
 .../admin/internal/SystemMemberImpl.java        |   21 +-
 .../internal/SystemMemberRegionEventImpl.java   |   21 +-
 .../admin/internal/SystemMemberRegionImpl.java  |   21 +-
 .../internal/SystemMembershipEventImpl.java     |   21 +-
 .../gemfire/admin/internal/package.html         |   16 +
 .../com/gemstone/gemfire/admin/jmx/Agent.java   |   21 +-
 .../gemstone/gemfire/admin/jmx/AgentConfig.java |   21 +-
 .../gemfire/admin/jmx/AgentFactory.java         |   21 +-
 .../internal/AdminDistributedSystemJmxImpl.java |   21 +-
 .../admin/jmx/internal/AgentConfigImpl.java     |   21 +-
 .../gemfire/admin/jmx/internal/AgentImpl.java   |   35 +-
 .../admin/jmx/internal/AgentLauncher.java       |   21 +-
 .../admin/jmx/internal/CacheServerJmxImpl.java  |   21 +-
 .../admin/jmx/internal/ConfigAttributeInfo.java |   20 +-
 .../internal/ConfigurationParameterJmxImpl.java |   21 +-
 .../DistributedSystemHealthConfigJmxImpl.java   |   21 +-
 .../internal/DistributionLocatorJmxImpl.java    |   22 +-
 .../admin/jmx/internal/DynamicManagedBean.java  |   20 +-
 .../internal/GemFireHealthConfigJmxImpl.java    |   35 +-
 .../jmx/internal/GemFireHealthJmxImpl.java      |   21 +-
 .../admin/jmx/internal/GenerateMBeanHTML.java   |   21 +-
 .../gemfire/admin/jmx/internal/MBeanUtil.java   |   20 +-
 .../admin/jmx/internal/MX4JModelMBean.java      |   21 +-
 .../jmx/internal/MX4JServerSocketFactory.java   |   22 +-
 .../gemfire/admin/jmx/internal/MailManager.java |   21 +-
 .../admin/jmx/internal/ManagedResource.java     |   21 +-
 .../admin/jmx/internal/ManagedResourceType.java |   21 +-
 .../jmx/internal/MemberInfoWithStatsMBean.java  |   21 +-
 .../admin/jmx/internal/RMIRegistryService.java  |   20 +-
 .../jmx/internal/RMIRegistryServiceMBean.java   |   20 +-
 .../jmx/internal/RefreshNotificationType.java   |   21 +-
 .../jmx/internal/StatAlertNotification.java     |   21 +-
 .../jmx/internal/StatAlertsAggregator.java      |   21 +-
 .../jmx/internal/StatisticAttributeInfo.java    |   20 +-
 .../jmx/internal/StatisticResourceJmxImpl.java  |   21 +-
 .../SystemMemberBridgeServerJmxImpl.java        |   21 +-
 .../jmx/internal/SystemMemberCacheJmxImpl.java  |   34 +-
 .../admin/jmx/internal/SystemMemberJmx.java     |   21 +-
 .../admin/jmx/internal/SystemMemberJmxImpl.java |   21 +-
 .../jmx/internal/SystemMemberRegionJmxImpl.java |   21 +-
 .../gemfire/admin/jmx/internal/package.html     |   16 +
 .../com/gemstone/gemfire/admin/jmx/package.html |   16 +
 .../com/gemstone/gemfire/admin/package.html     |   16 +
 .../gemfire/cache/AttributesFactory.java        |  101 +-
 .../gemfire/cache/AttributesMutator.java        |   21 +-
 .../java/com/gemstone/gemfire/cache/Cache.java  |   50 +-
 .../gemstone/gemfire/cache/CacheCallback.java   |   21 +-
 .../gemfire/cache/CacheClosedException.java     |   21 +-
 .../com/gemstone/gemfire/cache/CacheEvent.java  |   21 +-
 .../gemstone/gemfire/cache/CacheException.java  |   21 +-
 .../gemfire/cache/CacheExistsException.java     |   21 +-
 .../gemstone/gemfire/cache/CacheFactory.java    |   21 +-
 .../gemstone/gemfire/cache/CacheListener.java   |   21 +-
 .../com/gemstone/gemfire/cache/CacheLoader.java |   21 +-
 .../gemfire/cache/CacheLoaderException.java     |   21 +-
 .../gemfire/cache/CacheRuntimeException.java    |   21 +-
 .../gemstone/gemfire/cache/CacheStatistics.java |   21 +-
 .../gemfire/cache/CacheTransactionManager.java  |   21 +-
 .../com/gemstone/gemfire/cache/CacheWriter.java |   21 +-
 .../gemfire/cache/CacheWriterException.java     |   21 +-
 .../gemfire/cache/CacheXmlException.java        |   21 +-
 .../gemstone/gemfire/cache/ClientSession.java   |   21 +-
 .../gemfire/cache/CommitConflictException.java  |   21 +-
 .../cache/CommitDistributionException.java      |   21 +-
 .../cache/CommitIncompleteException.java        |   16 +
 .../gemfire/cache/CustomEvictionAttributes.java |   22 +-
 .../gemstone/gemfire/cache/CustomExpiry.java    |   21 +-
 .../com/gemstone/gemfire/cache/DataPolicy.java  |   40 +-
 .../com/gemstone/gemfire/cache/Declarable.java  |   21 +-
 .../gemfire/cache/DiskAccessException.java      |   21 +-
 .../com/gemstone/gemfire/cache/DiskStore.java   |   21 +-
 .../gemfire/cache/DiskStoreFactory.java         |   36 +-
 .../gemfire/cache/DiskWriteAttributes.java      |   21 +-
 .../cache/DiskWriteAttributesFactory.java       |   21 +-
 .../DuplicatePrimaryPartitionException.java     |   21 +-
 .../gemfire/cache/DynamicRegionFactory.java     |  142 +-
 .../gemfire/cache/DynamicRegionListener.java    |   21 +-
 .../gemfire/cache/EntryDestroyedException.java  |   21 +-
 .../com/gemstone/gemfire/cache/EntryEvent.java  |   21 +-
 .../gemfire/cache/EntryExistsException.java     |   21 +-
 .../gemfire/cache/EntryNotFoundException.java   |   21 +-
 .../gemfire/cache/EntryNotFoundInRegion.java    |   21 +-
 .../gemstone/gemfire/cache/EntryOperation.java  |   23 +-
 .../gemstone/gemfire/cache/EvictionAction.java  |   26 +-
 .../gemfire/cache/EvictionAlgorithm.java        |   26 +-
 .../gemfire/cache/EvictionAttributes.java       |   20 +-
 .../cache/EvictionAttributesMutator.java        |   23 +-
 .../gemfire/cache/EvictionCriteria.java         |   22 +-
 .../gemfire/cache/ExpirationAction.java         |   21 +-
 .../gemfire/cache/ExpirationAttributes.java     |   21 +-
 .../cache/FailedSynchronizationException.java   |   21 +-
 .../gemfire/cache/FixedPartitionAttributes.java |   21 +-
 .../gemfire/cache/FixedPartitionResolver.java   |   20 +-
 .../cache/GatewayConfigurationException.java    |   21 +-
 .../gemfire/cache/GatewayException.java         |   21 +-
 .../gemstone/gemfire/cache/GemFireCache.java    |   46 +-
 .../cache/IncompatibleVersionException.java     |   21 +-
 .../gemstone/gemfire/cache/InterestPolicy.java  |   21 +-
 .../cache/InterestRegistrationEvent.java        |   21 +-
 .../cache/InterestRegistrationListener.java     |   21 +-
 .../gemfire/cache/InterestResultPolicy.java     |   21 +-
 .../gemstone/gemfire/cache/LoaderHelper.java    |   21 +-
 .../com/gemstone/gemfire/cache/LossAction.java  |   21 +-
 .../gemfire/cache/LowMemoryException.java       |   21 +-
 .../gemfire/cache/MembershipAttributes.java     |   21 +-
 .../com/gemstone/gemfire/cache/MirrorType.java  |   21 +-
 .../cache/NoQueueServersAvailableException.java |   21 +-
 ...NoSubscriptionServersAvailableException.java |   21 +-
 .../com/gemstone/gemfire/cache/Operation.java   |   21 +-
 .../cache/OperationAbortedException.java        |   21 +-
 .../gemfire/cache/PartitionAttributes.java      |   21 +-
 .../cache/PartitionAttributesFactory.java       |   20 +-
 .../gemfire/cache/PartitionResolver.java        |   20 +-
 .../PartitionedRegionDistributionException.java |   21 +-
 .../PartitionedRegionStorageException.java      |   21 +-
 .../java/com/gemstone/gemfire/cache/Region.java |   21 +-
 .../gemfire/cache/RegionAccessException.java    |   21 +-
 .../gemfire/cache/RegionAttributes.java         |   21 +-
 .../gemfire/cache/RegionDestroyedException.java |   21 +-
 .../cache/RegionDistributionException.java      |   21 +-
 .../com/gemstone/gemfire/cache/RegionEvent.java |   21 +-
 .../gemfire/cache/RegionExistsException.java    |   21 +-
 .../gemstone/gemfire/cache/RegionFactory.java   |   46 +-
 .../gemfire/cache/RegionMembershipListener.java |   21 +-
 .../cache/RegionReinitializedException.java     |   21 +-
 .../gemfire/cache/RegionRoleException.java      |   21 +-
 .../gemfire/cache/RegionRoleListener.java       |   21 +-
 .../gemstone/gemfire/cache/RegionService.java   |   25 +-
 .../gemstone/gemfire/cache/RegionShortcut.java  |   71 +-
 .../cache/RemoteTransactionException.java       |   21 +-
 .../gemstone/gemfire/cache/RequiredRoles.java   |   21 +-
 .../gemfire/cache/ResourceException.java        |   21 +-
 .../gemfire/cache/ResumptionAction.java         |   21 +-
 .../com/gemstone/gemfire/cache/RoleEvent.java   |   21 +-
 .../gemstone/gemfire/cache/RoleException.java   |   21 +-
 .../java/com/gemstone/gemfire/cache/Scope.java  |   21 +-
 .../gemfire/cache/SerializedCacheValue.java     |   21 +-
 .../cache/StatisticsDisabledException.java      |   21 +-
 .../gemfire/cache/SubscriptionAttributes.java   |   21 +-
 .../SynchronizationCommitConflictException.java |   21 +-
 .../gemfire/cache/TimeoutException.java         |   21 +-
 ...TransactionDataNodeHasDepartedException.java |   21 +-
 .../TransactionDataNotColocatedException.java   |   21 +-
 .../TransactionDataRebalancedException.java     |   21 +-
 .../gemfire/cache/TransactionEvent.java         |   21 +-
 .../gemfire/cache/TransactionException.java     |   21 +-
 .../gemstone/gemfire/cache/TransactionId.java   |   21 +-
 .../cache/TransactionInDoubtException.java      |   21 +-
 .../gemfire/cache/TransactionListener.java      |   21 +-
 .../gemfire/cache/TransactionWriter.java        |   21 +-
 .../cache/TransactionWriterException.java       |   21 +-
 ...upportedOperationInTransactionException.java |   21 +-
 .../cache/UnsupportedVersionException.java      |   21 +-
 .../gemfire/cache/VersionException.java         |   21 +-
 .../gemfire/cache/asyncqueue/AsyncEvent.java    |   21 +-
 .../cache/asyncqueue/AsyncEventListener.java    |   21 +-
 .../cache/asyncqueue/AsyncEventQueue.java       |   21 +-
 .../asyncqueue/AsyncEventQueueFactory.java      |   23 +-
 .../internal/AsyncEventQueueFactoryImpl.java    |   30 +-
 .../internal/AsyncEventQueueImpl.java           |   24 +-
 .../internal/AsyncEventQueueStats.java          |   21 +-
 .../internal/ParallelAsyncEventQueueImpl.java   |   18 +-
 .../internal/SerialAsyncEventQueueImpl.java     |   18 +-
 .../client/AllConnectionsInUseException.java    |   21 +-
 .../gemfire/cache/client/ClientCache.java       |   21 +-
 .../cache/client/ClientCacheFactory.java        |   39 +-
 .../cache/client/ClientNotReadyException.java   |   21 +-
 .../cache/client/ClientRegionFactory.java       |   21 +-
 .../cache/client/ClientRegionShortcut.java      |   23 +-
 .../client/NoAvailableLocatorsException.java    |   21 +-
 .../client/NoAvailableServersException.java     |   21 +-
 .../com/gemstone/gemfire/cache/client/Pool.java |   21 +-
 .../gemfire/cache/client/PoolFactory.java       |   49 +-
 .../gemfire/cache/client/PoolManager.java       |   21 +-
 .../client/ServerConnectivityException.java     |   21 +-
 .../cache/client/ServerOperationException.java  |   21 +-
 .../ServerRefusedConnectionException.java       |   21 +-
 .../client/SubscriptionNotEnabledException.java |   21 +-
 .../client/doc-files/example-client-cache.xml   |   16 +
 .../cache/client/internal/AbstractOp.java       |   23 +-
 .../cache/client/internal/AddPDXEnumOp.java     |   21 +-
 .../cache/client/internal/AddPDXTypeOp.java     |   21 +-
 .../client/internal/AuthenticateUserOp.java     |   23 +-
 .../internal/AutoConnectionSourceImpl.java      |   21 +-
 .../cache/client/internal/BridgePoolImpl.java   |  479 --
 .../internal/BridgeServerLoadMessage.java       |   99 -
 .../client/internal/CacheServerLoadMessage.java |  108 +
 .../gemfire/cache/client/internal/ClearOp.java  |   21 +-
 .../client/internal/ClientMetadataService.java  |   20 +-
 .../client/internal/ClientPartitionAdvisor.java |   21 +-
 .../internal/ClientRegionFactoryImpl.java       |   21 +-
 .../cache/client/internal/ClientUpdater.java    |   21 +-
 .../client/internal/CloseConnectionOp.java      |   21 +-
 .../gemfire/cache/client/internal/CommitOp.java |   21 +-
 .../cache/client/internal/Connection.java       |   21 +-
 .../client/internal/ConnectionFactory.java      |   21 +-
 .../client/internal/ConnectionFactoryImpl.java  |   52 +-
 .../cache/client/internal/ConnectionImpl.java   |   53 +-
 .../cache/client/internal/ConnectionSource.java |   21 +-
 .../cache/client/internal/ConnectionStats.java  |   21 +-
 .../cache/client/internal/ContainsKeyOp.java    |   21 +-
 .../DataSerializerRecoveryListener.java         |   21 +-
 .../cache/client/internal/DestroyOp.java        |   26 +-
 .../cache/client/internal/DestroyRegionOp.java  |   21 +-
 .../gemfire/cache/client/internal/Endpoint.java |   21 +-
 .../cache/client/internal/EndpointManager.java  |   21 +-
 .../client/internal/EndpointManagerImpl.java    |   31 +-
 .../cache/client/internal/ExecutablePool.java   |   21 +-
 .../client/internal/ExecuteFunctionHelper.java  |   21 +-
 .../client/internal/ExecuteFunctionNoAckOp.java |   21 +-
 .../client/internal/ExecuteFunctionOp.java      |   21 +-
 .../internal/ExecuteRegionFunctionNoAckOp.java  |   21 +-
 .../internal/ExecuteRegionFunctionOp.java       |   21 +-
 .../ExecuteRegionFunctionSingleHopOp.java       |   21 +-
 .../internal/ExplicitConnectionSourceImpl.java  |  106 +-
 .../gemfire/cache/client/internal/GetAllOp.java |   21 +-
 .../client/internal/GetClientPRMetaDataOp.java  |   20 +-
 .../GetClientPartitionAttributesOp.java         |   20 +-
 .../cache/client/internal/GetEntryOp.java       |   21 +-
 .../cache/client/internal/GetEventValueOp.java  |   20 +-
 .../client/internal/GetFunctionAttributeOp.java |   21 +-
 .../gemfire/cache/client/internal/GetOp.java    |   26 +-
 .../cache/client/internal/GetPDXEnumByIdOp.java |   21 +-
 .../cache/client/internal/GetPDXEnumsOp.java    |   21 +-
 .../client/internal/GetPDXIdForEnumOp.java      |   21 +-
 .../client/internal/GetPDXIdForTypeOp.java      |   21 +-
 .../cache/client/internal/GetPDXTypeByIdOp.java |   21 +-
 .../cache/client/internal/GetPDXTypesOp.java    |   21 +-
 .../internal/InstantiatorRecoveryListener.java  |   21 +-
 .../cache/client/internal/InternalPool.java     |   21 +-
 .../cache/client/internal/InvalidateOp.java     |   21 +-
 .../gemfire/cache/client/internal/KeySetOp.java |   21 +-
 .../cache/client/internal/LiveServerPinger.java |   21 +-
 .../internal/LocatorDiscoveryCallback.java      |   21 +-
 .../LocatorDiscoveryCallbackAdapter.java        |   21 +-
 .../cache/client/internal/MakePrimaryOp.java    |   21 +-
 .../gemfire/cache/client/internal/Op.java       |   23 +-
 .../cache/client/internal/OpExecutorImpl.java   |   21 +-
 .../internal/PdxRegistryRecoveryListener.java   |   21 +-
 .../gemfire/cache/client/internal/PingOp.java   |   37 +-
 .../gemfire/cache/client/internal/PoolImpl.java |   35 +-
 .../cache/client/internal/PrimaryAckOp.java     |   21 +-
 .../cache/client/internal/ProxyCache.java       |   21 +-
 .../client/internal/ProxyCacheCloseOp.java      |   21 +-
 .../cache/client/internal/ProxyRegion.java      |   21 +-
 .../gemfire/cache/client/internal/PutAllOp.java |   21 +-
 .../gemfire/cache/client/internal/PutOp.java    |   26 +-
 .../gemfire/cache/client/internal/QueryOp.java  |   21 +-
 .../client/internal/QueueConnectionImpl.java    |   21 +-
 .../cache/client/internal/QueueManager.java     |   21 +-
 .../cache/client/internal/QueueManagerImpl.java |   37 +-
 .../cache/client/internal/QueueState.java       |   16 +
 .../cache/client/internal/QueueStateImpl.java   |   27 +-
 .../cache/client/internal/ReadyForEventsOp.java |   21 +-
 .../internal/RegisterDataSerializersOp.java     |   29 +-
 .../internal/RegisterInstantiatorsOp.java       |   29 +-
 .../client/internal/RegisterInterestListOp.java |   21 +-
 .../client/internal/RegisterInterestOp.java     |   21 +-
 .../internal/RegisterInterestTracker.java       |   21 +-
 .../cache/client/internal/RemoveAllOp.java      |   21 +-
 .../cache/client/internal/RollbackOp.java       |   21 +-
 .../cache/client/internal/ServerBlackList.java  |   21 +-
 .../cache/client/internal/ServerProxy.java      |   21 +-
 .../client/internal/ServerRegionDataAccess.java |   23 +-
 .../client/internal/ServerRegionProxy.java      |   65 +-
 .../internal/SingleHopClientExecutor.java       |   21 +-
 .../internal/SingleHopOperationCallable.java    |   21 +-
 .../gemfire/cache/client/internal/SizeOp.java   |   21 +-
 .../cache/client/internal/TXFailoverOp.java     |   21 +-
 .../client/internal/TXSynchronizationOp.java    |   21 +-
 .../internal/UnregisterInterestListOp.java      |   21 +-
 .../client/internal/UnregisterInterestOp.java   |   21 +-
 .../cache/client/internal/UserAttributes.java   |   21 +-
 .../locator/ClientConnectionRequest.java        |   21 +-
 .../locator/ClientConnectionResponse.java       |   21 +-
 .../locator/ClientReplacementRequest.java       |   21 +-
 .../internal/locator/GetAllServersRequest.java  |   20 +-
 .../internal/locator/GetAllServersResponse.java |   20 +-
 .../internal/locator/LocatorListRequest.java    |   21 +-
 .../internal/locator/LocatorListResponse.java   |   21 +-
 .../internal/locator/LocatorStatusRequest.java  |   20 +-
 .../internal/locator/LocatorStatusResponse.java |   20 +-
 .../locator/QueueConnectionRequest.java         |   21 +-
 .../locator/QueueConnectionResponse.java        |   21 +-
 .../internal/locator/SerializationHelper.java   |   21 +-
 .../internal/locator/ServerLocationRequest.java |   23 +-
 .../locator/ServerLocationResponse.java         |   23 +-
 .../locator/wan/LocatorMembershipListener.java  |   21 +-
 .../gemfire/cache/client/internal/package.html  |   16 +
 .../pooling/ConnectionDestroyedException.java   |   21 +-
 .../internal/pooling/ConnectionManager.java     |   21 +-
 .../internal/pooling/ConnectionManagerImpl.java |   30 +-
 .../internal/pooling/PooledConnection.java      |   21 +-
 .../gemstone/gemfire/cache/client/package.html  |   15 +
 .../gemfire/cache/control/RebalanceFactory.java |   21 +-
 .../cache/control/RebalanceOperation.java       |   21 +-
 .../gemfire/cache/control/RebalanceResults.java |   21 +-
 .../gemfire/cache/control/ResourceManager.java  |   21 +-
 .../gemstone/gemfire/cache/control/package.html |   16 +
 .../gemfire/cache/doc-files/example-cache.xml   |   16 +
 .../gemfire/cache/doc-files/example2-cache.xml  |   17 +
 .../gemfire/cache/doc-files/example3-cache.xml  |   16 +
 .../execute/EmtpyRegionFunctionException.java   |   21 +-
 .../gemfire/cache/execute/Execution.java        |   20 +-
 .../gemfire/cache/execute/Function.java         |   20 +-
 .../gemfire/cache/execute/FunctionAdapter.java  |   20 +-
 .../gemfire/cache/execute/FunctionContext.java  |   20 +-
 .../cache/execute/FunctionException.java        |   20 +-
 .../FunctionInvocationTargetException.java      |   20 +-
 .../gemfire/cache/execute/FunctionService.java  |   20 +-
 .../cache/execute/RegionFunctionContext.java    |   21 +-
 .../gemfire/cache/execute/ResultCollector.java  |   20 +-
 .../gemfire/cache/execute/ResultSender.java     |   20 +-
 .../internal/FunctionServiceManager.java        |   27 +-
 .../gemstone/gemfire/cache/execute/package.html |   15 +
 .../cache/hdfs/HDFSEventQueueAttributes.java    |   72 -
 .../hdfs/HDFSEventQueueAttributesFactory.java   |  160 -
 .../gemfire/cache/hdfs/HDFSIOException.java     |   21 +-
 .../gemstone/gemfire/cache/hdfs/HDFSStore.java  |  402 +-
 .../gemfire/cache/hdfs/HDFSStoreFactory.java    |  292 +-
 .../gemfire/cache/hdfs/HDFSStoreMutator.java    |  336 +-
 .../cache/hdfs/StoreExistsException.java        |   21 +-
 .../cache/hdfs/internal/FailureTracker.java     |   21 +-
 .../cache/hdfs/internal/FlushObserver.java      |   16 +
 .../hdfs/internal/HDFSBucketRegionQueue.java    |   21 +-
 .../cache/hdfs/internal/HDFSEntriesSet.java     |   21 +-
 .../cache/hdfs/internal/HDFSEventListener.java  |   25 +-
 .../internal/HDFSEventQueueAttributesImpl.java  |  179 -
 .../hdfs/internal/HDFSEventQueueFilter.java     |   21 +-
 .../hdfs/internal/HDFSGatewayEventImpl.java     |   22 +-
 .../hdfs/internal/HDFSIntegrationUtil.java      |  142 +-
 .../HDFSParallelGatewaySenderQueue.java         |   21 +-
 .../hdfs/internal/HDFSStoreConfigHolder.java    |  681 ++-
 .../cache/hdfs/internal/HDFSStoreCreation.java  |  212 +-
 .../hdfs/internal/HDFSStoreFactoryImpl.java     |   23 +-
 .../cache/hdfs/internal/HDFSStoreImpl.java      |  108 +-
 .../hdfs/internal/HDFSStoreMutatorImpl.java     |  302 +-
 .../HDFSWriteOnlyStoreEventListener.java        |   21 +-
 .../hdfs/internal/HoplogListenerForRegion.java  |   21 +-
 .../cache/hdfs/internal/PersistedEventImpl.java |   21 +-
 .../hdfs/internal/QueuedPersistentEvent.java    |   16 +
 .../hdfs/internal/SignalledFlushObserver.java   |   16 +
 .../SizeTieredHdfsCompactionConfigHolder.java   |   74 -
 .../internal/SortedHDFSQueuePersistedEvent.java |   21 +-
 .../internal/SortedHoplogPersistedEvent.java    |   21 +-
 .../UnsortedHDFSQueuePersistedEvent.java        |   21 +-
 .../internal/UnsortedHoplogPersistedEvent.java  |   21 +-
 .../cache/hdfs/internal/cardinality/Bits.java   |   21 +-
 .../cardinality/CardinalityMergeException.java  |   21 +-
 .../hdfs/internal/cardinality/HyperLogLog.java  |   21 +-
 .../hdfs/internal/cardinality/IBuilder.java     |   21 +-
 .../hdfs/internal/cardinality/ICardinality.java |   21 +-
 .../hdfs/internal/cardinality/MurmurHash.java   |   21 +-
 .../hdfs/internal/cardinality/RegisterSet.java  |   21 +-
 .../hdfs/internal/hoplog/AbstractHoplog.java    |   21 +-
 .../hoplog/AbstractHoplogOrganizer.java         |   21 +-
 .../cache/hdfs/internal/hoplog/BloomFilter.java |   21 +-
 .../hoplog/CloseTmpHoplogsTimerTask.java        |   21 +-
 .../hdfs/internal/hoplog/CompactionStatus.java  |   21 +-
 .../cache/hdfs/internal/hoplog/FlushStatus.java |   16 +
 .../internal/hoplog/HDFSCompactionManager.java  |   48 +-
 .../internal/hoplog/HDFSFlushQueueArgs.java     |   16 +
 .../internal/hoplog/HDFSFlushQueueFunction.java |   16 +
 .../hoplog/HDFSForceCompactionArgs.java         |   21 +-
 .../hoplog/HDFSForceCompactionFunction.java     |   21 +-
 .../HDFSForceCompactionResultCollector.java     |   21 +-
 .../hoplog/HDFSLastCompactionTimeFunction.java  |   21 +-
 .../internal/hoplog/HDFSRegionDirector.java     |   21 +-
 .../hdfs/internal/hoplog/HDFSStoreDirector.java |   21 +-
 .../hoplog/HDFSUnsortedHoplogOrganizer.java     |   31 +-
 .../hdfs/internal/hoplog/HFileSortedOplog.java  |   21 +-
 .../hoplog/HdfsSortedOplogOrganizer.java        |   43 +-
 .../cache/hdfs/internal/hoplog/Hoplog.java      |   21 +-
 .../hdfs/internal/hoplog/HoplogConfig.java      |   21 +-
 .../hdfs/internal/hoplog/HoplogListener.java    |   21 +-
 .../hdfs/internal/hoplog/HoplogOrganizer.java   |   21 +-
 .../hdfs/internal/hoplog/HoplogSetIterator.java |   21 +-
 .../hdfs/internal/hoplog/HoplogSetReader.java   |   21 +-
 .../internal/hoplog/SequenceFileHoplog.java     |   21 +-
 .../hoplog/mapred/AbstractGFRecordReader.java   |   21 +-
 .../internal/hoplog/mapred/GFInputFormat.java   |   21 +-
 .../internal/hoplog/mapred/GFOutputFormat.java  |   21 +-
 .../mapreduce/AbstractGFRecordReader.java       |   21 +-
 .../hoplog/mapreduce/GFInputFormat.java         |   21 +-
 .../hdfs/internal/hoplog/mapreduce/GFKey.java   |   21 +-
 .../hoplog/mapreduce/GFOutputFormat.java        |   21 +-
 .../hoplog/mapreduce/HDFSSplitIterator.java     |   21 +-
 .../internal/hoplog/mapreduce/HoplogUtil.java   |   21 +-
 .../hoplog/mapreduce/RWSplitIterator.java       |   21 +-
 .../hoplog/mapreduce/StreamSplitIterator.java   |   21 +-
 .../org/apache/hadoop/io/SequenceFile.java      |   21 +-
 .../gemfire/cache/lucene/LuceneIndex.java       |   50 -
 .../gemfire/cache/lucene/LuceneQuery.java       |   37 -
 .../cache/lucene/LuceneQueryFactory.java        |  121 -
 .../cache/lucene/LuceneQueryResults.java        |   29 -
 .../cache/lucene/LuceneResultStruct.java        |   59 -
 .../gemfire/cache/lucene/LuceneService.java     |  103 -
 .../cache/lucene/LuceneServiceFactory.java      |   14 -
 .../cache/lucene/LuceneServiceProvider.java     |   36 -
 .../cache/lucene/internal/LuceneIndexImpl.java  |   54 -
 .../lucene/internal/LuceneQueryFactoryImpl.java |   88 -
 .../cache/lucene/internal/LuceneQueryImpl.java  |   62 -
 .../lucene/internal/LuceneQueryResultsImpl.java |   66 -
 .../lucene/internal/LuceneResultStructImpl.java |   45 -
 .../internal/LuceneServiceFactoryImpl.java      |   16 -
 .../lucene/internal/LuceneServiceImpl.java      |   90 -
 .../operations/CloseCQOperationContext.java     |   21 +-
 .../operations/DestroyOperationContext.java     |   21 +-
 .../operations/ExecuteCQOperationContext.java   |   21 +-
 .../ExecuteFunctionOperationContext.java        |   21 +-
 .../GetDurableCQsOperationContext.java          |   21 +-
 .../cache/operations/GetOperationContext.java   |   28 +-
 .../operations/InterestOperationContext.java    |   21 +-
 .../gemfire/cache/operations/InterestType.java  |   21 +-
 .../operations/InvalidateOperationContext.java  |   21 +-
 .../cache/operations/KeyOperationContext.java   |   21 +-
 .../operations/KeySetOperationContext.java      |   21 +-
 .../operations/KeyValueOperationContext.java    |   91 +-
 .../cache/operations/OperationContext.java      |   21 +-
 .../operations/PutAllOperationContext.java      |   21 +-
 .../cache/operations/PutOperationContext.java   |   21 +-
 .../cache/operations/QueryOperationContext.java |   21 +-
 .../operations/RegionClearOperationContext.java |   21 +-
 .../RegionCreateOperationContext.java           |   21 +-
 .../RegionDestroyOperationContext.java          |   21 +-
 .../operations/RegionOperationContext.java      |   21 +-
 .../RegisterInterestOperationContext.java       |   21 +-
 .../operations/RemoveAllOperationContext.java   |   21 +-
 .../operations/StopCQOperationContext.java      |   21 +-
 .../UnregisterInterestOperationContext.java     |   21 +-
 .../internal/GetOperationContextImpl.java       |   34 +-
 .../gemfire/cache/operations/package.html       |   16 +
 .../com/gemstone/gemfire/cache/package.html     |   16 +
 .../cache/partition/PartitionListener.java      |   20 +-
 .../partition/PartitionListenerAdapter.java     |   23 +-
 .../cache/partition/PartitionManager.java       |  369 --
 .../cache/partition/PartitionMemberInfo.java    |   21 +-
 .../PartitionNotAvailableException.java         |   21 +-
 .../cache/partition/PartitionRebalanceInfo.java |   21 +-
 .../cache/partition/PartitionRegionHelper.java  |   20 +-
 .../cache/partition/PartitionRegionInfo.java    |   21 +-
 .../gemfire/cache/partition/package.html        |   16 +
 .../ConflictingPersistentDataException.java     |   21 +-
 .../persistence/PartitionOfflineException.java  |   21 +-
 .../gemfire/cache/persistence/PersistentID.java |   23 +-
 .../PersistentReplicatesOfflineException.java   |   21 +-
 .../persistence/RevokeFailedException.java      |   21 +-
 .../RevokedPersistentDataException.java         |   21 +-
 .../gemfire/cache/query/Aggregator.java         |   16 +
 .../cache/query/AmbiguousNameException.java     |   21 +-
 .../gemfire/cache/query/CqAttributes.java       |   21 +-
 .../cache/query/CqAttributesFactory.java        |   21 +-
 .../cache/query/CqAttributesMutator.java        |   21 +-
 .../gemfire/cache/query/CqClosedException.java  |   21 +-
 .../gemstone/gemfire/cache/query/CqEvent.java   |   21 +-
 .../gemfire/cache/query/CqException.java        |   21 +-
 .../gemfire/cache/query/CqExistsException.java  |   21 +-
 .../gemfire/cache/query/CqListener.java         |   21 +-
 .../gemstone/gemfire/cache/query/CqQuery.java   |   21 +-
 .../gemstone/gemfire/cache/query/CqResults.java |   21 +-
 .../cache/query/CqServiceStatistics.java        |   21 +-
 .../gemstone/gemfire/cache/query/CqState.java   |   21 +-
 .../gemfire/cache/query/CqStatistics.java       |   21 +-
 .../gemfire/cache/query/CqStatusListener.java   |   21 +-
 .../cache/query/FunctionDomainException.java    |   21 +-
 .../com/gemstone/gemfire/cache/query/Index.java |   21 +-
 .../cache/query/IndexCreationException.java     |   21 +-
 .../cache/query/IndexExistsException.java       |   21 +-
 .../cache/query/IndexInvalidException.java      |   21 +-
 .../cache/query/IndexMaintenanceException.java  |   21 +-
 .../cache/query/IndexNameConflictException.java |   21 +-
 .../gemfire/cache/query/IndexStatistics.java    |   21 +-
 .../gemstone/gemfire/cache/query/IndexType.java |   21 +-
 .../query/MultiIndexCreationException.java      |   16 +
 .../cache/query/NameNotFoundException.java      |   21 +-
 .../cache/query/NameResolutionException.java    |   21 +-
 .../query/ParameterCountInvalidException.java   |   21 +-
 .../com/gemstone/gemfire/cache/query/Query.java |   21 +-
 .../gemfire/cache/query/QueryException.java     |   21 +-
 .../query/QueryExecutionLowMemoryException.java |   21 +-
 .../query/QueryExecutionTimeoutException.java   |   21 +-
 .../cache/query/QueryInvalidException.java      |   21 +-
 .../query/QueryInvocationTargetException.java   |   21 +-
 .../gemfire/cache/query/QueryService.java       |   21 +-
 .../gemfire/cache/query/QueryStatistics.java    |   21 +-
 .../cache/query/RegionNotFoundException.java    |   21 +-
 .../gemfire/cache/query/SelectResults.java      |   21 +-
 .../gemstone/gemfire/cache/query/Struct.java    |   21 +-
 .../cache/query/TypeMismatchException.java      |   21 +-
 .../query/internal/AbstractCompiledValue.java   |   22 +-
 .../internal/AbstractGroupOrRangeJunction.java  |   21 +-
 .../cache/query/internal/AllGroupJunction.java  |   21 +-
 .../query/internal/AttributeDescriptor.java     |   22 +-
 .../gemfire/cache/query/internal/Bag.java       |   21 +-
 .../internal/CompiledAggregateFunction.java     |   16 +
 .../query/internal/CompiledBindArgument.java    |   22 +-
 .../query/internal/CompiledComparison.java      |   22 +-
 .../query/internal/CompiledConstruction.java    |   22 +-
 .../cache/query/internal/CompiledFunction.java  |   22 +-
 .../query/internal/CompiledGroupBySelect.java   |   16 +
 .../cache/query/internal/CompiledID.java        |   22 +-
 .../cache/query/internal/CompiledIn.java        |   22 +-
 .../query/internal/CompiledIndexOperation.java  |   22 +-
 .../query/internal/CompiledIteratorDef.java     |   21 +-
 .../cache/query/internal/CompiledJunction.java  |   22 +-
 .../cache/query/internal/CompiledLike.java      |   21 +-
 .../cache/query/internal/CompiledLiteral.java   |   22 +-
 .../cache/query/internal/CompiledNegation.java  |   22 +-
 .../cache/query/internal/CompiledOperation.java |   26 +-
 .../cache/query/internal/CompiledPath.java      |   22 +-
 .../cache/query/internal/CompiledRegion.java    |   22 +-
 .../cache/query/internal/CompiledSelect.java    |   61 +-
 .../query/internal/CompiledSortCriterion.java   |   21 +-
 .../query/internal/CompiledUnaryMinus.java      |   21 +-
 .../cache/query/internal/CompiledUndefined.java |   22 +-
 .../cache/query/internal/CompiledValue.java     |   22 +-
 .../query/internal/CompositeGroupJunction.java  |   21 +-
 .../gemfire/cache/query/internal/CqEntry.java   |   21 +-
 .../cache/query/internal/CqQueryVsdStats.java   |   21 +-
 .../cache/query/internal/CqStateImpl.java       |   22 +-
 .../internal/CumulativeNonDistinctResults.java  |   16 +
 .../cache/query/internal/DefaultQuery.java      |   31 +-
 .../query/internal/DefaultQueryService.java     |   22 +-
 .../cache/query/internal/ExecutionContext.java  |   22 +-
 .../gemfire/cache/query/internal/Filter.java    |   22 +-
 .../gemfire/cache/query/internal/Functions.java |   22 +-
 .../cache/query/internal/GroupJunction.java     |   21 +-
 .../cache/query/internal/HashingStrategy.java   |   21 +-
 .../gemfire/cache/query/internal/IndexInfo.java |   21 +-
 .../internal/IndexTrackingQueryObserver.java    |   28 +-
 .../cache/query/internal/IndexUpdater.java      |   21 +-
 .../gemfire/cache/query/internal/Indexable.java |   21 +-
 .../cache/query/internal/LinkedResultSet.java   |   21 +-
 .../cache/query/internal/LinkedStructSet.java   |   21 +-
 .../cache/query/internal/MapIndexable.java      |   16 +
 .../cache/query/internal/MethodDispatch.java    |   22 +-
 .../cache/query/internal/NWayMergeResults.java  |   16 +
 .../gemfire/cache/query/internal/Negatable.java |   22 +-
 .../gemfire/cache/query/internal/NullToken.java |   23 +-
 .../cache/query/internal/ObjectIntHashMap.java  |   21 +-
 .../cache/query/internal/OrderByComparator.java |   18 +-
 .../internal/OrderByComparatorUnmapped.java     |   16 +
 .../gemfire/cache/query/internal/Ordered.java   |   16 +
 .../cache/query/internal/OrganizedOperands.java |   21 +-
 .../cache/query/internal/PRQueryTraceInfo.java  |   22 +-
 .../gemfire/cache/query/internal/PathUtils.java |   22 +-
 .../gemfire/cache/query/internal/PlanInfo.java  |   22 +-
 .../cache/query/internal/ProxyQueryService.java |   21 +-
 .../gemfire/cache/query/internal/QCompiler.java |   22 +-
 .../gemfire/cache/query/internal/QRegion.java   |   21 +-
 .../gemfire/cache/query/internal/QScope.java    |   22 +-
 .../QueryExecutionCanceledException.java        |   21 +-
 .../query/internal/QueryExecutionContext.java   |   24 +-
 .../cache/query/internal/QueryExecutor.java     |   21 +-
 .../cache/query/internal/QueryMonitor.java      |   21 +-
 .../cache/query/internal/QueryObserver.java     |   22 +-
 .../query/internal/QueryObserverAdapter.java    |   22 +-
 .../query/internal/QueryObserverHolder.java     |   22 +-
 .../cache/query/internal/QueryUtils.java        |   21 +-
 .../cache/query/internal/RangeJunction.java     |   21 +-
 .../cache/query/internal/ResultsBag.java        |   16 +
 .../ResultsCollectionCopyOnReadWrapper.java     |   21 +-
 ...ResultsCollectionPdxDeserializerWrapper.java |   21 +-
 .../internal/ResultsCollectionWrapper.java      |   21 +-
 .../cache/query/internal/ResultsSet.java        |   21 +-
 .../cache/query/internal/RuntimeIterator.java   |   25 +-
 .../query/internal/SelectResultsComparator.java |   21 +-
 .../cache/query/internal/SortedResultSet.java   |   21 +-
 .../cache/query/internal/SortedResultsBag.java  |   16 +
 .../cache/query/internal/SortedStructBag.java   |   16 +
 .../cache/query/internal/SortedStructSet.java   |   21 +-
 .../gemfire/cache/query/internal/StructBag.java |   21 +-
 .../cache/query/internal/StructFields.java      |   16 +
 .../cache/query/internal/StructImpl.java        |   21 +-
 .../gemfire/cache/query/internal/StructSet.java |   21 +-
 .../gemfire/cache/query/internal/Support.java   |   22 +-
 .../gemfire/cache/query/internal/Undefined.java |   22 +-
 .../internal/aggregate/AbstractAggregator.java  |   16 +
 .../cache/query/internal/aggregate/Avg.java     |   16 +
 .../query/internal/aggregate/AvgBucketNode.java |   16 +
 .../query/internal/aggregate/AvgDistinct.java   |   16 +
 .../aggregate/AvgDistinctPRQueryNode.java       |   16 +
 .../internal/aggregate/AvgPRQueryNode.java      |   16 +
 .../cache/query/internal/aggregate/Count.java   |   16 +
 .../query/internal/aggregate/CountDistinct.java |   16 +
 .../aggregate/CountDistinctPRQueryNode.java     |   16 +
 .../internal/aggregate/CountPRQueryNode.java    |   16 +
 .../internal/aggregate/DistinctAggregator.java  |   16 +
 .../cache/query/internal/aggregate/MaxMin.java  |   16 +
 .../cache/query/internal/aggregate/Sum.java     |   16 +
 .../query/internal/aggregate/SumDistinct.java   |   16 +
 .../aggregate/SumDistinctPRQueryNode.java       |   16 +
 .../cache/query/internal/cq/ClientCQ.java       |   16 +
 .../cache/query/internal/cq/CqService.java      |   16 +
 .../query/internal/cq/CqServiceProvider.java    |   16 +
 .../query/internal/cq/InternalCqQuery.java      |   16 +
 .../query/internal/cq/MissingCqService.java     |   16 +
 .../internal/cq/MissingCqServiceStatistics.java |   16 +
 .../cache/query/internal/cq/ServerCQ.java       |   16 +
 .../query/internal/cq/spi/CqServiceFactory.java |   16 +
 .../query/internal/index/AbstractIndex.java     |   50 +-
 .../query/internal/index/AbstractMapIndex.java  |   27 +-
 .../internal/index/CompactMapRangeIndex.java    |   28 +-
 .../query/internal/index/CompactRangeIndex.java |   21 +-
 .../query/internal/index/DummyQRegion.java      |   23 +-
 .../index/FunctionalIndexCreationHelper.java    |   22 +-
 .../cache/query/internal/index/HashIndex.java   |  190 +-
 .../query/internal/index/HashIndexSet.java      | 1107 ++---
 .../query/internal/index/HashIndexStrategy.java |   81 -
 .../query/internal/index/IMQException.java      |   21 +-
 .../internal/index/IndexConcurrentHashSet.java  |   21 +-
 .../query/internal/index/IndexCreationData.java |   21 +-
 .../internal/index/IndexCreationHelper.java     |   21 +-
 .../cache/query/internal/index/IndexData.java   |   21 +-
 .../query/internal/index/IndexElemArray.java    |   21 +-
 .../query/internal/index/IndexManager.java      |   24 +-
 .../query/internal/index/IndexProtocol.java     |   20 +-
 .../cache/query/internal/index/IndexStats.java  |   21 +-
 .../cache/query/internal/index/IndexStore.java  |   23 +-
 .../cache/query/internal/index/IndexUtils.java  |   21 +-
 .../index/IndexedExpressionEvaluator.java       |   21 +-
 .../query/internal/index/MapIndexStore.java     |   21 +-
 .../query/internal/index/MapRangeIndex.java     |   21 +-
 .../query/internal/index/MemoryIndexStore.java  |   21 +-
 .../query/internal/index/PartitionedIndex.java  |   37 +-
 .../query/internal/index/PrimaryKeyIndex.java   |   21 +-
 .../index/PrimaryKeyIndexCreationHelper.java    |   21 +-
 .../cache/query/internal/index/RangeIndex.java  |   21 +-
 .../cache/query/internal/index/package.html     |   16 +
 .../gemfire/cache/query/internal/package.html   |   16 +
 .../query/internal/parse/ASTAggregateFunc.java  |   16 +
 .../cache/query/internal/parse/ASTAnd.java      |   21 +-
 .../query/internal/parse/ASTCombination.java    |   21 +-
 .../query/internal/parse/ASTCompareOp.java      |   21 +-
 .../query/internal/parse/ASTConstruction.java   |   21 +-
 .../query/internal/parse/ASTConversionExpr.java |   21 +-
 .../cache/query/internal/parse/ASTDummy.java    |   16 +
 .../cache/query/internal/parse/ASTGroupBy.java  |   21 +-
 .../cache/query/internal/parse/ASTHint.java     |   21 +-
 .../query/internal/parse/ASTHintIdentifier.java |   21 +-
 .../query/internal/parse/ASTIdentifier.java     |   21 +-
 .../cache/query/internal/parse/ASTImport.java   |   21 +-
 .../cache/query/internal/parse/ASTIn.java       |   21 +-
 .../query/internal/parse/ASTIteratorDef.java    |   21 +-
 .../cache/query/internal/parse/ASTLike.java     |   21 +-
 .../cache/query/internal/parse/ASTLimit.java    |   23 +-
 .../cache/query/internal/parse/ASTLiteral.java  |   21 +-
 .../internal/parse/ASTMethodInvocation.java     |   21 +-
 .../cache/query/internal/parse/ASTOr.java       |   21 +-
 .../cache/query/internal/parse/ASTOrderBy.java  |   21 +-
 .../query/internal/parse/ASTParameter.java      |   21 +-
 .../cache/query/internal/parse/ASTPostfix.java  |   21 +-
 .../query/internal/parse/ASTProjection.java     |   21 +-
 .../query/internal/parse/ASTRegionPath.java     |   21 +-
 .../cache/query/internal/parse/ASTSelect.java   |   21 +-
 .../query/internal/parse/ASTSortCriterion.java  |   21 +-
 .../cache/query/internal/parse/ASTTrace.java    |   21 +-
 .../cache/query/internal/parse/ASTType.java     |   21 +-
 .../cache/query/internal/parse/ASTTypeCast.java |   21 +-
 .../cache/query/internal/parse/ASTUnary.java    |   21 +-
 .../query/internal/parse/ASTUndefinedExpr.java  |   21 +-
 .../query/internal/parse/ASTUnsupported.java    |   21 +-
 .../cache/query/internal/parse/GemFireAST.java  |   21 +-
 .../cache/query/internal/parse/UtilParser.java  |   21 +-
 .../cache/query/internal/parse/fixantlr.sh      |   15 +
 .../gemfire/cache/query/internal/parse/oql.g    |   17 +
 .../internal/types/CollectionTypeImpl.java      |   21 +-
 .../types/ExtendedNumericComparator.java        |   22 +-
 .../cache/query/internal/types/MapTypeImpl.java |   21 +-
 .../query/internal/types/NumericComparator.java |   22 +-
 .../query/internal/types/ObjectTypeImpl.java    |   21 +-
 .../query/internal/types/StructTypeImpl.java    |   21 +-
 .../internal/types/TemporalComparator.java      |   22 +-
 .../cache/query/internal/types/TypeUtils.java   |   22 +-
 .../query/internal/utils/LimitIterator.java     |   16 +
 .../cache/query/internal/utils/PDXUtils.java    |   16 +
 .../gemstone/gemfire/cache/query/package.html   |   16 +
 .../cache/query/types/CollectionType.java       |   21 +-
 .../gemfire/cache/query/types/MapType.java      |   21 +-
 .../gemfire/cache/query/types/ObjectType.java   |   21 +-
 .../gemfire/cache/query/types/StructType.java   |   21 +-
 .../gemfire/cache/server/CacheServer.java       |   21 +-
 .../cache/server/ClientSubscriptionConfig.java  |   21 +-
 .../gemfire/cache/server/ServerLoad.java        |   21 +-
 .../gemfire/cache/server/ServerLoadProbe.java   |   31 +-
 .../cache/server/ServerLoadProbeAdapter.java    |   21 +-
 .../gemfire/cache/server/ServerMetrics.java     |   31 +-
 .../server/internal/ConnectionCountProbe.java   |   21 +-
 .../cache/server/internal/LoadMonitor.java      |   33 +-
 .../server/internal/ServerMetricsImpl.java      |   21 +-
 .../gemstone/gemfire/cache/server/package.html  |   15 +
 .../cache/snapshot/CacheSnapshotService.java    |   21 +-
 .../cache/snapshot/RegionSnapshotService.java   |   21 +-
 .../gemfire/cache/snapshot/SnapshotFilter.java  |   21 +-
 .../cache/snapshot/SnapshotIterator.java        |   21 +-
 .../gemfire/cache/snapshot/SnapshotOptions.java |   21 +-
 .../gemfire/cache/snapshot/SnapshotReader.java  |   21 +-
 .../gemfire/cache/snapshot/package.html         |   17 +-
 .../cache/util/BoundedLinkedHashMap.java        |   24 +-
 .../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 -
 .../cache/util/CacheListenerAdapter.java        |   21 +-
 .../gemfire/cache/util/CacheWriterAdapter.java  |   21 +-
 .../gemfire/cache/util/CqListenerAdapter.java   |   21 +-
 .../util/EndpointDoesNotExistException.java     |   34 -
 .../gemfire/cache/util/EndpointException.java   |   61 -
 .../cache/util/EndpointExistsException.java     |   28 -
 .../cache/util/EndpointInUseException.java      |   27 -
 .../gemstone/gemfire/cache/util/Gateway.java    |   16 +
 .../cache/util/GatewayConflictHelper.java       |   21 +-
 .../cache/util/GatewayConflictResolver.java     |   21 +-
 .../gemfire/cache/util/GatewayEvent.java        |   21 +-
 .../util/IncompatibleVersionException.java      |   47 -
 .../gemfire/cache/util/ObjectSizer.java         |   21 +-
 .../gemfire/cache/util/ObjectSizerImpl.java     |   16 +
 .../util/RegionMembershipListenerAdapter.java   |   21 +-
 .../cache/util/RegionRoleListenerAdapter.java   |   21 +-
 .../util/ServerRefusedConnectionException.java  |   36 -
 .../cache/util/TimestampedEntryEvent.java       |   21 +-
 .../cache/util/TransactionListenerAdapter.java  |   21 +-
 .../UniversalMembershipListenerAdapter.java     |  352 --
 .../cache/util/UnknownVersionException.java     |   39 -
 .../gemfire/cache/util/VersionException.java    |   45 -
 .../gemstone/gemfire/cache/util/package.html    |   18 +-
 .../gemfire/cache/wan/EventSequenceID.java      |   21 +-
 .../gemfire/cache/wan/GatewayEventFilter.java   |   20 +-
 .../wan/GatewayEventSubstitutionFilter.java     |   21 +-
 .../gemfire/cache/wan/GatewayQueueEvent.java    |   21 +-
 .../gemfire/cache/wan/GatewayReceiver.java      |   20 +-
 .../cache/wan/GatewayReceiverFactory.java       |   20 +-
 .../gemfire/cache/wan/GatewaySender.java        |   20 +-
 .../gemfire/cache/wan/GatewaySenderFactory.java |   20 +-
 .../cache/wan/GatewayTransportFilter.java       |   20 +-
 .../compression/CompressionException.java       |   23 +-
 .../gemfire/compression/Compressor.java         |   23 +-
 .../gemfire/compression/SnappyCompressor.java   |   21 +-
 .../gemfire/distributed/AbstractLauncher.java   |   22 +-
 .../distributed/ClientSocketFactory.java        |   21 +-
 .../distributed/DistributedLockService.java     |   21 +-
 .../gemfire/distributed/DistributedMember.java  |   21 +-
 .../gemfire/distributed/DistributedSystem.java  |  312 +-
 .../DistributedSystemDisconnectedException.java |   23 +-
 .../distributed/DurableClientAttributes.java    |   21 +-
 .../distributed/FutureCancelledException.java   |   21 +-
 .../distributed/GatewayCancelledException.java  |   23 +-
 .../distributed/LeaseExpiredException.java      |   21 +-
 .../gemstone/gemfire/distributed/Locator.java   |   26 +-
 .../gemfire/distributed/LocatorLauncher.java    |   22 +-
 .../distributed/LockNotHeldException.java       |   21 +-
 .../LockServiceDestroyedException.java          |   21 +-
 .../distributed/OplogCancelledException.java    |   23 +-
 .../distributed/PoolCancelledException.java     |   23 +-
 .../com/gemstone/gemfire/distributed/Role.java  |   21 +-
 .../gemfire/distributed/ServerLauncher.java     |   26 +-
 .../TXManagerCancelledException.java            |   23 +-
 .../internal/AbstractDistributionConfig.java    |   42 +-
 .../distributed/internal/AdminMessageType.java  |   16 +
 .../internal/AtomicLongWithTerminalState.java   |   21 +-
 .../internal/CollectingReplyProcessor.java      |   21 +-
 .../distributed/internal/ConflationKey.java     |   21 +-
 .../gemfire/distributed/internal/DM.java        |   30 +-
 .../gemfire/distributed/internal/DMStats.java   |  145 +-
 .../gemfire/distributed/internal/DSClock.java   |   18 +-
 .../internal/DirectReplyProcessor.java          |   21 +-
 .../distributed/internal/DistributedState.java  |   21 +-
 .../internal/DistributionAdvisee.java           |   21 +-
 .../internal/DistributionAdvisor.java           |   23 +-
 .../internal/DistributionChannel.java           |   25 +-
 .../internal/DistributionConfig.java            |   41 +-
 .../internal/DistributionConfigImpl.java        |   52 +-
 .../internal/DistributionConfigSnapshot.java    |   21 +-
 .../internal/DistributionException.java         |   21 +-
 .../internal/DistributionManager.java           |  590 +--
 .../internal/DistributionManagerConfig.java     |   93 -
 .../internal/DistributionMessage.java           |   38 +-
 .../internal/DistributionMessageObserver.java   |   23 +-
 .../distributed/internal/DistributionStats.java |  357 +-
 .../distributed/internal/FlowControlParams.java |   21 +-
 .../internal/ForceDisconnectOperation.java      |   21 +-
 .../FunctionExecutionPooledExecutor.java        |   28 +-
 .../distributed/internal/HealthMonitor.java     |   21 +-
 .../distributed/internal/HealthMonitorImpl.java |   21 +-
 .../internal/HighPriorityAckedMessage.java      |   29 +-
 .../HighPriorityDistributionMessage.java        |   21 +-
 .../distributed/internal/IgnoredByManager.java  |   16 +
 .../internal/InternalDistributedSystem.java     |  202 +-
 .../distributed/internal/InternalLocator.java   |  172 +-
 .../internal/LocatorLoadSnapshot.java           |   20 +-
 .../distributed/internal/LocatorStats.java      |   21 +-
 .../internal/LonerDistributionManager.java      |  261 +-
 .../gemfire/distributed/internal/MQueue.java    |   16 +
 .../internal/MembershipListener.java            |   24 +-
 .../distributed/internal/MessageFactory.java    |   21 +-
 .../distributed/internal/MessageWithReply.java  |   21 +-
 .../internal/OverflowQueueWithDMStats.java      |   21 +-
 .../distributed/internal/PoolStatHelper.java    |   21 +-
 .../internal/PooledDistributionMessage.java     |   21 +-
 .../internal/PooledExecutorWithDMStats.java     |   38 +-
 .../distributed/internal/ProcessorKeeper21.java |   21 +-
 .../distributed/internal/ProductUseLog.java     |   23 +-
 .../distributed/internal/ProfileListener.java   |   21 +-
 .../distributed/internal/QueueStatHelper.java   |   21 +-
 .../internal/ReliableReplyException.java        |   21 +-
 .../internal/ReliableReplyProcessor21.java      |   21 +-
 .../distributed/internal/ReplyException.java    |   21 +-
 .../distributed/internal/ReplyMessage.java      |   23 +-
 .../distributed/internal/ReplyProcessor21.java  |   30 +-
 .../distributed/internal/ReplySender.java       |   21 +-
 .../distributed/internal/ResourceEvent.java     |   20 +-
 .../internal/ResourceEventsListener.java        |   20 +-
 .../internal/RuntimeDistributionConfigImpl.java |   21 +-
 .../internal/SerialAckedMessage.java            |   21 +-
 .../internal/SerialDistributionMessage.java     |   21 +-
 .../SerialQueuedExecutorWithDMStats.java        |   21 +-
 .../distributed/internal/ServerLocation.java    |   21 +-
 .../distributed/internal/ServerLocator.java     |   31 +-
 .../internal/SharedConfiguration.java           |   21 +-
 .../distributed/internal/ShutdownMessage.java   |   23 +-
 .../gemfire/distributed/internal/Sizeable.java  |   16 +
 .../distributed/internal/SizeableRunnable.java  |   21 +-
 .../distributed/internal/StartupMessage.java    |   56 +-
 .../internal/StartupMessageData.java            |   21 +-
 .../internal/StartupMessageReplyProcessor.java  |   21 +-
 .../distributed/internal/StartupOperation.java  |   27 +-
 .../internal/StartupResponseMessage.java        |   23 +-
 .../StartupResponseWithVersionMessage.java      |   23 +-
 .../internal/ThrottledMemQueueStatHelper.java   |   21 +-
 .../internal/ThrottledQueueStatHelper.java      |   21 +-
 .../ThrottlingMemLinkedQueueWithDMStats.java    |   21 +-
 .../internal/WaitForViewInstallation.java       |   25 +-
 .../internal/WanLocatorDiscoverer.java          |   16 +
 .../deadlock/DLockDependencyMonitor.java        |   21 +-
 .../internal/deadlock/DeadlockDetector.java     |  206 +-
 .../internal/deadlock/Dependency.java           |   21 +-
 .../internal/deadlock/DependencyGraph.java      |  193 +-
 .../internal/deadlock/DependencyMonitor.java    |   21 +-
 .../deadlock/DependencyMonitorManager.java      |   21 +-
 .../deadlock/GemFireDeadlockDetector.java       |   21 +-
 .../internal/deadlock/LocalLockInfo.java        |   23 +-
 .../internal/deadlock/LocalThread.java          |   25 +-
 .../deadlock/MessageDependencyMonitor.java      |   25 +-
 .../internal/deadlock/ThreadReference.java      |   21 +-
 .../internal/deadlock/UnsafeThreadLocal.java    |   21 +-
 .../internal/direct/DirectChannel.java          |  142 +-
 .../internal/direct/DirectChannelListener.java  |   38 +
 .../internal/direct/MissingStubException.java   |   28 -
 .../internal/direct/ShunnedMemberException.java |   34 +
 .../internal/distribution-overview.html         |   16 +
 .../internal/locks/Collaboration.java           |   21 +-
 .../distributed/internal/locks/DLockBatch.java  |   21 +-
 .../internal/locks/DLockBatchId.java            |   21 +-
 .../internal/locks/DLockGrantor.java            |   28 +-
 .../locks/DLockLessorDepartureHandler.java      |   21 +-
 .../internal/locks/DLockQueryProcessor.java     |   21 +-
 .../locks/DLockRecoverGrantorProcessor.java     |   21 +-
 .../internal/locks/DLockReleaseProcessor.java   |   21 +-
 .../internal/locks/DLockRemoteToken.java        |   21 +-
 .../internal/locks/DLockRequestProcessor.java   |   21 +-
 .../internal/locks/DLockService.java            |   21 +-
 .../distributed/internal/locks/DLockStats.java  |   21 +-
 .../distributed/internal/locks/DLockToken.java  |   21 +-
 .../internal/locks/DeposeGrantorProcessor.java  |   21 +-
 .../internal/locks/DistributedLockStats.java    |   21 +-
 .../internal/locks/DistributedMemberLock.java   |   21 +-
 .../internal/locks/DummyDLockStats.java         |   21 +-
 .../internal/locks/ElderInitProcessor.java      |   21 +-
 .../distributed/internal/locks/ElderState.java  |   21 +-
 .../distributed/internal/locks/GrantorInfo.java |   21 +-
 .../internal/locks/GrantorRequestProcessor.java |   21 +-
 .../locks/LockGrantorDestroyedException.java    |   21 +-
 .../internal/locks/LockGrantorId.java           |   21 +-
 .../locks/NonGrantorDestroyedProcessor.java     |   21 +-
 .../internal/locks/RemoteThread.java            |   21 +-
 .../distributed/internal/locks/package.html     |   16 +
 .../DistributedMembershipListener.java          |   37 +-
 .../membership/InternalDistributedMember.java   |  362 +-
 .../internal/membership/InternalRole.java       |   21 +-
 .../internal/membership/MemberAttributes.java   |  172 +-
 .../internal/membership/MemberFactory.java      |   54 +-
 .../internal/membership/MemberServices.java     |   49 +-
 .../internal/membership/MembershipManager.java  |   94 +-
 .../internal/membership/MembershipTestHook.java |   21 +-
 .../internal/membership/NetMember.java          |   84 +-
 .../internal/membership/NetView.java            |  615 ++-
 .../internal/membership/QuorumChecker.java      |   33 +-
 .../internal/membership/gms/GMSMember.java      |  458 ++
 .../membership/gms/GMSMemberFactory.java        |  133 +
 .../internal/membership/gms/GMSUtil.java        |  159 +
 .../internal/membership/gms/NetLocator.java     |   32 +
 .../internal/membership/gms/ServiceConfig.java  |  187 +
 .../internal/membership/gms/Services.java       |  387 ++
 .../internal/membership/gms/SuspectMember.java  |   58 +
 .../membership/gms/auth/GMSAuthenticator.java   |  235 +
 .../membership/gms/fd/GMSHealthMonitor.java     | 1300 +++++
 .../gms/interfaces/Authenticator.java           |   27 +
 .../gms/interfaces/HealthMonitor.java           |   65 +
 .../membership/gms/interfaces/JoinLeave.java    |   67 +
 .../membership/gms/interfaces/Locator.java      |   32 +
 .../membership/gms/interfaces/Manager.java      |  121 +
 .../gms/interfaces/MessageHandler.java          |   30 +
 .../membership/gms/interfaces/Messenger.java    |   81 +
 .../membership/gms/interfaces/Service.java      |   83 +
 .../gms/locator/FindCoordinatorRequest.java     |  148 +
 .../gms/locator/FindCoordinatorResponse.java    |  162 +
 .../membership/gms/locator/GMSLocator.java      |  373 ++
 .../membership/gms/locator/GetViewRequest.java  |   49 +
 .../membership/gms/locator/GetViewResponse.java |   64 +
 .../gms/locator/PeerLocatorRequest.java         |   25 +
 .../membership/gms/membership/GMSJoinLeave.java | 2246 +++++++++
 .../membership/gms/messages/HasMemberID.java    |   25 +
 .../gms/messages/HeartbeatMessage.java          |   74 +
 .../gms/messages/HeartbeatRequestMessage.java   |   80 +
 .../gms/messages/InstallViewMessage.java        |  106 +
 .../gms/messages/JoinRequestMessage.java        |   97 +
 .../gms/messages/JoinResponseMessage.java       |  129 +
 .../gms/messages/LeaveRequestMessage.java       |   94 +
 .../gms/messages/NetworkPartitionMessage.java   |   44 +
 .../gms/messages/RemoveMemberMessage.java       |   96 +
 .../gms/messages/SuspectMembersMessage.java     |   91 +
 .../membership/gms/messages/SuspectRequest.java |   72 +
 .../membership/gms/messages/ViewAckMessage.java |  103 +
 .../gms/messenger/AddressManager.java           |  121 +
 .../membership/gms/messenger/GMSPingPonger.java |   65 +
 .../gms/messenger/GMSQuorumChecker.java         |  274 ++
 .../membership/gms/messenger/JGAddress.java     |  221 +
 .../gms/messenger/JGroupsMessenger.java         | 1085 +++++
 .../membership/gms/messenger/StatRecorder.java  |  162 +
 .../membership/gms/messenger/Transport.java     |  168 +
 .../gms/mgr/GMSMembershipManager.java           | 2631 ++++++++++
 .../membership/gms/mgr/LocalViewMessage.java    |   85 +
 .../internal/membership/gms/package.html        |   57 +
 .../membership/jgroup/GFJGBasicAdapter.java     |  602 ---
 .../membership/jgroup/GFJGPeerAdapter.java      |  484 --
 .../membership/jgroup/JGroupMember.java         |  251 -
 .../membership/jgroup/JGroupMemberFactory.java  |  119 -
 .../jgroup/JGroupMembershipManager.java         | 4232 ----------------
 .../internal/membership/jgroup/LocatorImpl.java |  474 --
 .../membership/jgroup/QuorumCheckerImpl.java    |  291 --
 .../internal/membership/jgroup/ViewMessage.java |   73 -
 .../internal/membership/jgroup/package.html     |   43 -
 .../gemfire/distributed/internal/package.html   |   16 +
 .../internal/streaming/StreamingOperation.java  |   21 +-
 .../internal/tcpserver/InfoRequest.java         |   20 +-
 .../internal/tcpserver/InfoResponse.java        |   20 +-
 .../internal/tcpserver/ShutdownRequest.java     |   20 +-
 .../internal/tcpserver/ShutdownResponse.java    |   20 +-
 .../internal/tcpserver/TcpClient.java           |   76 +-
 .../internal/tcpserver/TcpHandler.java          |   16 +
 .../internal/tcpserver/TcpServer.java           |  121 +-
 .../internal/tcpserver/VersionRequest.java      |   16 +
 .../internal/tcpserver/VersionResponse.java     |   16 +
 .../unsafe/RegisterSignalHandlerSupport.java    |   20 +-
 .../gemstone/gemfire/distributed/package.html   |   16 +
 .../gemstone/gemfire/i18n/LogWriterI18n.java    |   29 +-
 .../com/gemstone/gemfire/i18n/StringId.java     |  161 +
 .../com/gemstone/gemfire/i18n/StringIdImpl.java |  153 -
 .../gemfire/internal/AbstractConfig.java        |   31 +-
 .../internal/AbstractStatisticsFactory.java     |   21 +-
 .../gemfire/internal/ArchiveSplitter.java       |   21 +-
 .../com/gemstone/gemfire/internal/Assert.java   |   21 +-
 .../gemfire/internal/AvailablePort.java         |   51 +-
 .../com/gemstone/gemfire/internal/Banner.java   |   49 +-
 .../gemfire/internal/ByteArrayDataInput.java    |   21 +-
 .../internal/ByteBufferOutputStream.java        |   21 +-
 .../gemfire/internal/ByteBufferWriter.java      |   16 +
 .../gemfire/internal/ClassLoadUtil.java         |   21 +-
 .../gemfire/internal/ClassPathLoader.java       |   21 +-
 .../com/gemstone/gemfire/internal/Config.java   |   21 +-
 .../gemstone/gemfire/internal/ConfigSource.java |   21 +-
 .../gemfire/internal/ConnectionWatcher.java     |   41 +
 .../gemfire/internal/CopyOnWriteHashSet.java    |   21 +-
 .../com/gemstone/gemfire/internal/DSCODE.java   |   21 +-
 .../gemstone/gemfire/internal/DSFIDFactory.java |   73 +-
 .../internal/DSFIDNotFoundException.java        |   21 +-
 .../internal/DataSerializableFixedID.java       |   73 +-
 .../gemfire/internal/DistributionLocator.java   |   21 +-
 .../internal/DummyStatisticsFactory.java        |   21 +-
 .../gemfire/internal/DummyStatisticsImpl.java   |   21 +-
 .../gemfire/internal/ExternalizableDSFID.java   |   21 +-
 .../com/gemstone/gemfire/internal/FileUtil.java |   21 +-
 .../gemfire/internal/GemFireStatSampler.java    |   21 +-
 .../gemfire/internal/GemFireUtilLauncher.java   |   21 +-
 .../gemfire/internal/GemFireVersion.java        |   21 +-
 .../internal/GfeConsoleReaderFactory.java       |   20 +-
 .../gemfire/internal/HeapDataOutputStream.java  |   21 +-
 .../gemfire/internal/HistogramStats.java        |   21 +-
 .../gemfire/internal/HostStatHelper.java        |   21 +-
 .../gemfire/internal/HostStatSampler.java       |   25 +-
 .../InsufficientDiskSpaceException.java         |   21 +-
 .../internal/InternalDataSerializer.java        |  203 +-
 .../gemfire/internal/InternalEntity.java        |   16 +
 .../gemfire/internal/InternalInstantiator.java  |   23 +-
 .../InternalStatisticsDisabledException.java    |   21 +-
 .../gemfire/internal/JarClassLoader.java        |   20 +-
 .../gemstone/gemfire/internal/JarDeployer.java  |   20 +-
 .../gemfire/internal/LinuxProcFsStatistics.java |   21 +-
 .../gemfire/internal/LinuxProcessStats.java     |   21 +-
 .../gemfire/internal/LinuxSystemStats.java      |   21 +-
 .../gemfire/internal/LocalStatListener.java     |   21 +-
 .../internal/LocalStatisticsFactory.java        |   21 +-
 .../gemfire/internal/LocalStatisticsImpl.java   |   21 +-
 .../gemstone/gemfire/internal/ManagerInfo.java  |   20 +-
 .../gemfire/internal/MigrationClient.java       |   21 +-
 .../gemfire/internal/MigrationServer.java       |   21 +-
 .../gemstone/gemfire/internal/NanoTimer.java    |   58 +-
 .../gemfire/internal/NullDataOutputStream.java  |   21 +-
 .../gemstone/gemfire/internal/OSProcess.java    |   19 +-
 .../gemfire/internal/OSXProcessStats.java       |   21 +-
 .../gemfire/internal/OSXSystemStats.java        |   21 +-
 .../gemfire/internal/ObjIdConcurrentMap.java    |   21 +-
 .../com/gemstone/gemfire/internal/ObjIdMap.java |   21 +-
 .../internal/ObjToByteArraySerializer.java      |   21 +-
 .../gemfire/internal/OneTaskOnlyExecutor.java   |   21 +-
 .../gemfire/internal/OsStatisticsFactory.java   |   21 +-
 .../gemfire/internal/PdxSerializerObject.java   |   21 +-
 .../gemfire/internal/ProcessOutputReader.java   |   21 +-
 .../gemstone/gemfire/internal/ProcessStats.java |   21 +-
 .../gemstone/gemfire/internal/PureJavaMode.java |   21 +-
 ...cheduledThreadPoolExecutorWithKeepAlive.java |   21 +-
 .../com/gemstone/gemfire/internal/Sendable.java |   18 +-
 .../gemfire/internal/SerializationVersions.java |   21 +-
 .../com/gemstone/gemfire/internal/SetUtils.java |   20 +-
 .../gemfire/internal/SharedLibrary.java         |   41 +-
 .../gemfire/internal/SimpleStatSampler.java     |   21 +-
 .../com/gemstone/gemfire/internal/SmHelper.java |   21 +-
 .../gemstone/gemfire/internal/SocketCloser.java |  257 +
 .../gemfire/internal/SocketCreator.java         |  162 +-
 .../gemfire/internal/SolarisProcessStats.java   |   21 +-
 .../gemfire/internal/SolarisSystemStats.java    |   21 +-
 .../gemfire/internal/StatArchiveFormat.java     |   21 +-
 .../gemfire/internal/StatArchiveReader.java     |   21 +-
 .../gemfire/internal/StatArchiveWriter.java     |   49 +-
 .../gemfire/internal/StatSamplerStats.java      |   21 +-
 .../internal/StatisticDescriptorImpl.java       |   21 +-
 .../gemfire/internal/StatisticsImpl.java        |   21 +-
 .../gemfire/internal/StatisticsManager.java     |   21 +-
 .../internal/StatisticsTypeFactoryImpl.java     |   21 +-
 .../gemfire/internal/StatisticsTypeImpl.java    |   21 +-
 .../gemfire/internal/StatisticsTypeXml.java     |   21 +-
 .../gemstone/gemfire/internal/SystemAdmin.java  |   36 +-
 .../gemfire/internal/SystemFailureTestHook.java |   21 +-
 .../gemstone/gemfire/internal/SystemTimer.java  |   21 +-
 .../gemfire/internal/UniqueIdGenerator.java     |   21 +-
 .../com/gemstone/gemfire/internal/VMStats.java  |   21 +-
 .../gemfire/internal/VMStatsContract.java       |   21 +-
 .../internal/VMStatsContractFactory.java        |   21 +-
 .../com/gemstone/gemfire/internal/Version.java  |   45 +-
 .../internal/VersionedDataInputStream.java      |   21 +-
 .../internal/VersionedDataOutputStream.java     |   21 +-
 .../internal/VersionedDataSerializable.java     |   16 +
 .../gemfire/internal/VersionedDataStream.java   |   21 +-
 .../gemfire/internal/VersionedObjectInput.java  |   21 +-
 .../gemfire/internal/VersionedObjectOutput.java |   21 +-
 .../gemfire/internal/WindowsProcessStats.java   |   21 +-
 .../gemfire/internal/WindowsSystemStats.java    |   21 +-
 .../internal/admin/AdminBridgeServer.java       |   16 +
 .../gemstone/gemfire/internal/admin/Alert.java  |   21 +-
 .../gemfire/internal/admin/AlertListener.java   |   21 +-
 .../gemfire/internal/admin/ApplicationVM.java   |   21 +-
 .../gemfire/internal/admin/CacheCollector.java  |   21 +-
 .../gemfire/internal/admin/CacheInfo.java       |   21 +-
 .../gemfire/internal/admin/CacheSnapshot.java   |   21 +-
 .../admin/ClientHealthMonitoringRegion.java     |   21 +-
 .../internal/admin/ClientMembershipMessage.java |   21 +-
 .../internal/admin/ClientStatsManager.java      |   28 +-
 .../internal/admin/CompoundEntrySnapshot.java   |   21 +-
 .../internal/admin/CompoundRegionSnapshot.java  |   21 +-
 .../gemfire/internal/admin/DLockInfo.java       |   21 +-
 .../gemfire/internal/admin/EntrySnapshot.java   |   21 +-
 .../gemfire/internal/admin/EntryValueNode.java  |   23 +-
 .../gemfire/internal/admin/GemFireVM.java       |   23 +-
 .../gemfire/internal/admin/GfManagerAgent.java  |   21 +-
 .../internal/admin/GfManagerAgentConfig.java    |   21 +-
 .../internal/admin/GfManagerAgentFactory.java   |   21 +-
 .../gemfire/internal/admin/GfObject.java        |   21 +-
 .../gemfire/internal/admin/HealthListener.java  |   21 +-
 .../internal/admin/JoinLeaveListener.java       |   21 +-
 .../gemfire/internal/admin/ListenerIdMap.java   |   21 +-
 .../gemfire/internal/admin/RegionSnapshot.java  |   21 +-
 .../gemfire/internal/admin/SSLConfig.java       |   21 +-
 .../gemfire/internal/admin/SnapshotClient.java  |   21 +-
 .../gemstone/gemfire/internal/admin/Stat.java   |   21 +-
 .../gemfire/internal/admin/StatAlert.java       |   21 +-
 .../internal/admin/StatAlertDefinition.java     |   21 +-
 .../internal/admin/StatAlertsManager.java       |   21 +-
 .../gemfire/internal/admin/StatListener.java    |   21 +-
 .../gemfire/internal/admin/StatResource.java    |   21 +-
 .../gemfire/internal/admin/TransportConfig.java |   21 +-
 .../gemfire/internal/admin/package.html         |   16 +
 .../admin/remote/AddHealthListenerRequest.java  |   21 +-
 .../admin/remote/AddHealthListenerResponse.java |   21 +-
 .../admin/remote/AddStatListenerRequest.java    |   21 +-
 .../admin/remote/AddStatListenerResponse.java   |   21 +-
 .../remote/AdminConsoleDisconnectMessage.java   |   21 +-
 .../admin/remote/AdminConsoleMessage.java       |   21 +-
 .../admin/remote/AdminFailureResponse.java      |   21 +-
 .../remote/AdminMultipleReplyProcessor.java     |   21 +-
 .../internal/admin/remote/AdminRegion.java      |   21 +-
 .../admin/remote/AdminReplyProcessor.java       |   21 +-
 .../internal/admin/remote/AdminRequest.java     |   21 +-
 .../internal/admin/remote/AdminResponse.java    |   23 +-
 .../internal/admin/remote/AdminWaiters.java     |   21 +-
 .../admin/remote/AlertLevelChangeMessage.java   |   26 +-
 .../admin/remote/AlertListenerMessage.java      |   23 +-
 .../admin/remote/AlertsNotificationMessage.java |   21 +-
 .../admin/remote/AppCacheSnapshotMessage.java   |   21 +-
 .../admin/remote/BridgeServerRequest.java       |   21 +-
 .../admin/remote/BridgeServerResponse.java      |   39 +-
 .../admin/remote/CacheConfigRequest.java        |   21 +-
 .../admin/remote/CacheConfigResponse.java       |   21 +-
 .../internal/admin/remote/CacheDisplay.java     |   21 +-
 .../internal/admin/remote/CacheInfoRequest.java |   21 +-
 .../admin/remote/CacheInfoResponse.java         |   21 +-
 .../admin/remote/CancelStatListenerRequest.java |   21 +-
 .../remote/CancelStatListenerResponse.java      |   21 +-
 .../internal/admin/remote/Cancellable.java      |   21 +-
 .../admin/remote/CancellationMessage.java       |   23 +-
 .../admin/remote/CancellationRegistry.java      |   23 +-
 .../remote/ChangeRefreshIntervalMessage.java    |   21 +-
 .../internal/admin/remote/CliLegacyMessage.java |   16 +
 .../admin/remote/ClientHealthStats.java         |   21 +-
 .../internal/admin/remote/CompactRequest.java   |   21 +-
 .../internal/admin/remote/CompactResponse.java  |   21 +-
 .../admin/remote/DestroyEntryMessage.java       |   23 +-
 .../admin/remote/DestroyRegionMessage.java      |   23 +-
 .../admin/remote/DistributionLocatorId.java     |   21 +-
 .../internal/admin/remote/DummyEntry.java       |   21 +-
 .../admin/remote/DurableClientInfoRequest.java  |   20 +-
 .../admin/remote/DurableClientInfoResponse.java |   24 +-
 .../admin/remote/EntryValueNodeImpl.java        |   23 +-
 .../admin/remote/FetchDistLockInfoRequest.java  |   21 +-
 .../admin/remote/FetchDistLockInfoResponse.java |   21 +-
 .../remote/FetchHealthDiagnosisRequest.java     |   21 +-
 .../remote/FetchHealthDiagnosisResponse.java    |   21 +-
 .../internal/admin/remote/FetchHostRequest.java |   21 +-
 .../admin/remote/FetchHostResponse.java         |   21 +-
 .../remote/FetchResourceAttributesRequest.java  |   21 +-
 .../remote/FetchResourceAttributesResponse.java |   21 +-
 .../admin/remote/FetchStatsRequest.java         |   21 +-
 .../admin/remote/FetchStatsResponse.java        |   22 +-
 .../admin/remote/FetchSysCfgRequest.java        |   23 +-
 .../admin/remote/FetchSysCfgResponse.java       |   21 +-
 .../remote/FlushAppCacheSnapshotMessage.java    |   21 +-
 .../admin/remote/HealthListenerMessage.java     |   23 +-
 .../remote/InspectionClasspathManager.java      |   21 +-
 .../admin/remote/LicenseInfoRequest.java        |   21 +-
 .../admin/remote/LicenseInfoResponse.java       |   21 +-
 .../remote/MissingPersistentIDsRequest.java     |   21 +-
 .../remote/MissingPersistentIDsResponse.java    |   21 +-
 .../admin/remote/ObjectDetailsRequest.java      |   21 +-
 .../admin/remote/ObjectDetailsResponse.java     |   21 +-
 .../admin/remote/ObjectNamesRequest.java        |   21 +-
 .../admin/remote/ObjectNamesResponse.java       |   21 +-
 .../PrepareRevokePersistentIDRequest.java       |   21 +-
 .../remote/RefreshMemberSnapshotRequest.java    |   21 +-
 .../remote/RefreshMemberSnapshotResponse.java   |   21 +-
 .../admin/remote/RegionAdminMessage.java        |   21 +-
 .../admin/remote/RegionAdminRequest.java        |   21 +-
 .../admin/remote/RegionAttributesRequest.java   |   21 +-
 .../admin/remote/RegionAttributesResponse.java  |   21 +-
 .../internal/admin/remote/RegionRequest.java    |   21 +-
 .../internal/admin/remote/RegionResponse.java   |   21 +-
 .../admin/remote/RegionSizeRequest.java         |   21 +-
 .../admin/remote/RegionSizeResponse.java        |   21 +-
 .../admin/remote/RegionStatisticsRequest.java   |   21 +-
 .../admin/remote/RegionStatisticsResponse.java  |   21 +-
 .../remote/RegionSubRegionSizeRequest.java      |   20 +-
 .../remote/RegionSubRegionsSizeResponse.java    |   20 +-
 .../internal/admin/remote/RemoteAlert.java      |   21 +-
 .../admin/remote/RemoteApplicationVM.java       |   21 +-
 .../admin/remote/RemoteBridgeServer.java        |   31 +-
 .../internal/admin/remote/RemoteCacheInfo.java  |   27 +-
 .../admin/remote/RemoteCacheStatistics.java     |   21 +-
 .../internal/admin/remote/RemoteDLockInfo.java  |   21 +-
 .../admin/remote/RemoteEntrySnapshot.java       |   21 +-
 .../internal/admin/remote/RemoteGemFireVM.java  |   23 +-
 .../admin/remote/RemoteGfManagerAgent.java      |   60 +-
 .../internal/admin/remote/RemoteObjectName.java |   21 +-
 .../admin/remote/RemoteRegionAttributes.java    |   23 +-
 .../admin/remote/RemoteRegionSnapshot.java      |   21 +-
 .../internal/admin/remote/RemoteStat.java       |   21 +-
 .../admin/remote/RemoteStatResource.java        |   21 +-
 .../admin/remote/RemoteTransportConfig.java     |  148 +-
 .../remote/RemoveHealthListenerRequest.java     |   21 +-
 .../remote/RemoveHealthListenerResponse.java    |   21 +-
 .../admin/remote/ResetHealthStatusRequest.java  |   21 +-
 .../admin/remote/ResetHealthStatusResponse.java |   21 +-
 .../admin/remote/RevokePersistentIDRequest.java |   21 +-
 .../remote/RevokePersistentIDResponse.java      |   21 +-
 .../admin/remote/RootRegionRequest.java         |   21 +-
 .../admin/remote/RootRegionResponse.java        |   21 +-
 .../remote/ShutdownAllGatewayHubsRequest.java   |   16 +
 .../admin/remote/ShutdownAllRequest.java        |   27 +-
 .../admin/remote/ShutdownAllResponse.java       |   23 +-
 .../admin/remote/SnapshotResultMessage.java     |   23 +-
 .../remote/StatAlertsManagerAssignMessage.java  |   21 +-
 .../admin/remote/StatListenerMessage.java       |   23 +-
 .../admin/remote/StoreSysCfgRequest.java        |   21 +-
 .../admin/remote/StoreSysCfgResponse.java       |   21 +-
 .../internal/admin/remote/SubRegionRequest.java |   21 +-
 .../admin/remote/SubRegionResponse.java         |   21 +-
 .../internal/admin/remote/TailLogRequest.java   |   21 +-
 .../internal/admin/remote/TailLogResponse.java  |   21 +-
 .../remote/UpdateAlertDefinitionMessage.java    |   21 +-
 .../admin/remote/VersionInfoRequest.java        |   21 +-
 .../admin/remote/VersionInfoResponse.java       |   21 +-
 .../admin/remote/VersionMismatchAlert.java      |   21 +-
 .../gemfire/internal/admin/remote/package.html  |   16 +
 .../admin/statalerts/BaseDecoratorImpl.java     |   21 +-
 .../statalerts/DummyStatisticInfoImpl.java      |   21 +-
 .../admin/statalerts/FunctionDecoratorImpl.java |   21 +-
 .../admin/statalerts/FunctionHelper.java        |   20 +-
 .../statalerts/GaugeThresholdDecoratorImpl.java |   21 +-
 .../statalerts/MultiAttrDefinitionImpl.java     |   21 +-
 .../NumberThresholdDecoratorImpl.java           |   21 +-
 .../statalerts/SingleAttrDefinitionImpl.java    |   21 +-
 .../admin/statalerts/StatisticInfo.java         |   21 +-
 .../admin/statalerts/StatisticInfoImpl.java     |   21 +-
 .../internal/cache/AbstractBridgeServer.java    |  425 --
 .../cache/AbstractBucketRegionQueue.java        |   21 +-
 .../internal/cache/AbstractCacheServer.java     |  407 ++
 .../cache/AbstractDiskLRURegionEntry.java       |   21 +-
 .../internal/cache/AbstractDiskRegion.java      |   21 +-
 .../internal/cache/AbstractDiskRegionEntry.java |   21 +-
 .../internal/cache/AbstractLRURegionEntry.java  |   21 +-
 .../internal/cache/AbstractLRURegionMap.java    |   21 +-
 .../cache/AbstractOplogDiskRegionEntry.java     |   21 +-
 .../gemfire/internal/cache/AbstractRegion.java  |  100 +-
 .../internal/cache/AbstractRegionEntry.java     |   52 +-
 .../internal/cache/AbstractRegionMap.java       |   37 +-
 .../internal/cache/AbstractUpdateOperation.java |   25 +-
 .../gemfire/internal/cache/AcceptHelper.java    |   21 +-
 .../cache/AddCacheServerProfileMessage.java     |   21 +-
 .../gemfire/internal/cache/BackupLock.java      |   21 +-
 .../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   |   23 +-
 .../gemfire/internal/cache/BucketDump.java      |   21 +-
 .../internal/cache/BucketNotFoundException.java |   21 +-
 .../cache/BucketPersistenceAdvisor.java         |   21 +-
 .../gemfire/internal/cache/BucketRegion.java    |   33 +-
 .../internal/cache/BucketRegionEvictior.java    |   21 +-
 .../internal/cache/BucketRegionQueue.java       |   21 +-
 .../internal/cache/BucketServerLocation.java    |   20 +-
 .../internal/cache/BucketServerLocation66.java  |   20 +-
 .../cache/BytesAndBitsForCompactor.java         |   23 +-
 .../internal/cache/CacheClientStatus.java       |   21 +-
 .../gemfire/internal/cache/CacheConfig.java     |   33 +-
 .../cache/CacheDistributionAdvisee.java         |   21 +-
 .../cache/CacheDistributionAdvisor.java         |   21 +-
 .../internal/cache/CacheLifecycleListener.java  |   21 +-
 .../gemfire/internal/cache/CacheObserver.java   |   26 +-
 .../internal/cache/CacheObserverAdapter.java    |   25 +-
 .../internal/cache/CacheObserverHolder.java     |   21 +-
 .../gemfire/internal/cache/CachePerfStats.java  |   21 +-
 .../internal/cache/CacheServerAdvisor.java      |  173 +
 .../gemfire/internal/cache/CacheServerImpl.java |  821 ++++
 .../internal/cache/CacheServerLauncher.java     |   21 +-
 .../gemfire/internal/cache/CacheService.java    |   42 +
 .../internal/cache/CacheStatisticsImpl.java     |   21 +-
 .../internal/cache/CachedDeserializable.java    |   21 +-
 .../cache/CachedDeserializableFactory.java      |   21 +-
 .../internal/cache/ClientRegionEventImpl.java   |  117 +
 .../internal/cache/ClientServerObserver.java    |   99 +
 .../cache/ClientServerObserverAdapter.java      |  116 +
 .../cache/ClientServerObserverHolder.java       |   62 +
 .../cache/ClientSubscriptionConfigImpl.java     |   21 +-
 .../internal/cache/CloseCacheMessage.java       |   23 +-
 .../cache/ClusterConfigurationLoader.java       |   16 +
 .../internal/cache/ColocationHelper.java        |   20 +-
 .../internal/cache/CommitReplyException.java    |   21 +-
 .../internal/cache/CompactableOplog.java        |   21 +-
 .../gemfire/internal/cache/Conflatable.java     |   21 +-
 .../internal/cache/ControllerAdvisor.java       |   21 +-
 .../internal/cache/CountingDataInputStream.java |   21 +-
 .../internal/cache/CreateRegionProcessor.java   |   27 +-
 .../internal/cache/CustomEntryExpiryTask.java   |   16 +
 .../cache/CustomEvictionAttributesImpl.java     |   21 +-
 .../internal/cache/DataLocationException.java   |   21 +-
 .../internal/cache/DestroyOperation.java        |   21 +-
 .../cache/DestroyPartitionedRegionMessage.java  |   21 +-
 .../internal/cache/DestroyRegionOperation.java  |   27 +-
 .../gemfire/internal/cache/DestroyedEntry.java  |   21 +-
 .../internal/cache/DirectReplyMessage.java      |   21 +-
 .../gemfire/internal/cache/DirectoryHolder.java |   21 +-
 .../internal/cache/DiskDirectoryStats.java      |   21 +-
 .../gemfire/internal/cache/DiskEntry.java       |   58 +-
 .../gemstone/gemfire/internal/cache/DiskId.java |   21 +-
 .../gemfire/internal/cache/DiskInitFile.java    |   21 +-
 .../gemfire/internal/cache/DiskRegion.java      |   30 +-
 .../gemfire/internal/cache/DiskRegionStats.java |   43 +-
 .../internal/cache/DiskStoreAttributes.java     |   21 +-
 .../gemfire/internal/cache/DiskStoreBackup.java |   21 +-
 .../internal/cache/DiskStoreFactoryImpl.java    |   21 +-
 .../gemfire/internal/cache/DiskStoreImpl.java   |   27 +-
 .../internal/cache/DiskStoreMonitor.java        |   21 +-
 .../internal/cache/DiskStoreObserver.java       |   21 +-
 .../gemfire/internal/cache/DiskStoreStats.java  |   21 +-
 .../gemfire/internal/cache/DiskStoreTask.java   |   21 +-
 .../internal/cache/DiskWriteAttributesImpl.java |   21 +-
 .../internal/cache/DistPeerTXStateStub.java     |   16 +
 .../cache/DistTXAdjunctCommitMessage.java       |   51 +
 .../internal/cache/DistTXCommitMessage.java     |   29 +-
 .../cache/DistTXCoordinatorInterface.java       |   21 +-
 .../internal/cache/DistTXPrecommitMessage.java  |   17 +-
 .../internal/cache/DistTXRollbackMessage.java   |   17 +-
 .../gemfire/internal/cache/DistTXState.java     |   93 +-
 .../cache/DistTXStateOnCoordinator.java         |   16 +
 .../internal/cache/DistTXStateProxyImpl.java    |   16 +
 .../DistTXStateProxyImplOnCoordinator.java      |   46 +-
 .../cache/DistTXStateProxyImplOnDatanode.java   |   16 +
 .../cache/DistributedCacheOperation.java        |   36 +-
 .../cache/DistributedClearOperation.java        |   27 +-
 .../cache/DistributedPutAllOperation.java       |   21 +-
 .../internal/cache/DistributedRegion.java       |   33 +-
 ...stributedRegionFunctionStreamingMessage.java |   21 +-
 .../cache/DistributedRemoveAllOperation.java    |   21 +-
 .../cache/DistributedTombstoneOperation.java    |   21 +-
 .../internal/cache/DummyCachePerfStats.java     |   21 +-
 .../internal/cache/DynamicRegionAttributes.java |   21 +-
 .../cache/DynamicRegionFactoryImpl.java         |   21 +-
 .../gemfire/internal/cache/EntriesMap.java      |   21 +-
 .../gemfire/internal/cache/EntriesSet.java      |   21 +-
 .../gemfire/internal/cache/EntryBits.java       |   21 +-
 .../gemfire/internal/cache/EntryEventImpl.java  |   75 +-
 .../gemfire/internal/cache/EntryExpiryTask.java |   24 +-
 .../internal/cache/EntryOperationImpl.java      |   23 +-
 .../gemfire/internal/cache/EntrySnapshot.java   |   23 +-
 .../internal/cache/EnumListenerEvent.java       |   20 +-
 .../gemfire/internal/cache/EventID.java         |   21 +-
 .../internal/cache/EventStateHelper.java        |   21 +-
 .../gemfire/internal/cache/EventTracker.java    |   21 +-
 .../internal/cache/EvictionAttributesImpl.java  |   23 +-
 .../gemfire/internal/cache/EvictorService.java  |   21 +-
 .../internal/cache/ExpirationScheduler.java     |   21 +-
 .../gemfire/internal/cache/ExpiryTask.java      |   89 +-
 .../internal/cache/ExportDiskRegion.java        |   16 +
 .../gemfire/internal/cache/FilterProfile.java   |   21 +-
 .../internal/cache/FilterRoutingInfo.java       |   21 +-
 .../cache/FindDurableQueueProcessor.java        |   25 +-
 .../internal/cache/FindRemoteTXMessage.java     |   23 +-
 .../internal/cache/FindVersionTagOperation.java |   21 +-
 .../cache/FixedPartitionAttributesImpl.java     |   21 +-
 .../internal/cache/ForceReattemptException.java |   21 +-
 .../cache/ForceableLinkedBlockingQueue.java     |   21 +-
 .../FunctionStreamingOrderedReplyMessage.java   |   21 +-
 .../cache/FunctionStreamingReplyMessage.java    |   21 +-
 .../internal/cache/GatewayEventFilter.java      |   16 +
 .../internal/cache/GemFireCacheImpl.java        |  260 +-
 .../internal/cache/GemfireCacheHelper.java      |   23 +-
 .../gemfire/internal/cache/GridAdvisor.java     |   27 +-
 .../gemfire/internal/cache/HARegion.java        |   21 +-
 .../internal/cache/HDFSLRURegionMap.java        |   21 +-
 .../gemfire/internal/cache/HDFSRegionMap.java   |   16 +
 .../internal/cache/HDFSRegionMapDelegate.java   |   21 +-
 .../internal/cache/HDFSRegionMapImpl.java       |   21 +-
 .../internal/cache/HasCachePerfStats.java       |   16 +
 .../gemfire/internal/cache/ImageState.java      |   21 +-
 .../cache/InMemoryPersistentMemberView.java     |   21 +-
 .../internal/cache/IncomingGatewayStatus.java   |   21 +-
 .../internal/cache/InitialImageFlowControl.java |   25 +-
 .../internal/cache/InitialImageOperation.java   |   21 +-
 .../gemfire/internal/cache/InlineKeyHelper.java |   21 +-
 .../gemfire/internal/cache/InterestEvent.java   |   21 +-
 .../gemfire/internal/cache/InterestFilter.java  |   21 +-
 .../cache/InterestRegistrationEventImpl.java    |   21 +-
 .../gemfire/internal/cache/InternalCache.java   |   23 +-
 .../internal/cache/InternalCacheEvent.java      |   21 +-
 .../internal/cache/InternalDataView.java        |   21 +-
 .../internal/cache/InternalRegionArguments.java |   21 +-
 .../internal/cache/InvalidateOperation.java     |   21 +-
 .../InvalidatePartitionedRegionMessage.java     |   21 +-
 .../cache/InvalidateRegionOperation.java        |   21 +-
 .../cache/JtaAfterCompletionMessage.java        |   21 +-
 .../cache/JtaBeforeCompletionMessage.java       |   21 +-
 .../gemfire/internal/cache/KeyInfo.java         |   21 +-
 .../internal/cache/KeyWithRegionContext.java    |   21 +-
 .../gemfire/internal/cache/ListOfDeltas.java    |   21 +-
 .../internal/cache/LoaderHelperFactory.java     |   21 +-
 .../internal/cache/LoaderHelperImpl.java        |   23 +-
 .../gemfire/internal/cache/LocalDataSet.java    |   21 +-
 .../gemfire/internal/cache/LocalRegion.java     |  133 +-
 .../internal/cache/LocalRegionDataView.java     |   21 +-
 .../cache/MemberFunctionStreamingMessage.java   |   30 +-
 .../cache/MinimumSystemRequirements.java        |   25 +-
 .../cache/NetSearchExpirationCalculator.java    |   21 +-
 .../gemstone/gemfire/internal/cache/Node.java   |   20 +-
 .../internal/cache/NonLocalRegionEntry.java     |   21 +-
 .../cache/NonLocalRegionEntryWithStats.java     |   23 +-
 .../internal/cache/OffHeapRegionEntry.java      |   16 +
 .../cache/OfflineCompactionDiskRegion.java      |   21 +-
 .../gemstone/gemfire/internal/cache/OpType.java |   21 +-
 .../gemstone/gemfire/internal/cache/Oplog.java  |   28 +-
 .../gemfire/internal/cache/OplogSet.java        |   16 +
 .../internal/cache/OrderedTombstoneMap.java     |   21 +-
 .../gemfire/internal/cache/OverflowOplog.java   |   21 +-
 .../internal/cache/OverflowOplogSet.java        |   21 +-
 .../internal/cache/PRContainsValueFunction.java |   21 +-
 .../internal/cache/PRHARedundancyProvider.java  |   26 +-
 .../internal/cache/PRQueryProcessor.java        |   20 +-
 .../internal/cache/PRSystemPropertyGetter.java  |   21 +-
 .../internal/cache/PartitionAttributesImpl.java |   21 +-
 .../internal/cache/PartitionRegionConfig.java   |   21 +-
 .../cache/PartitionRegionConfigValidator.java   |   21 +-
 .../internal/cache/PartitionedRegion.java       |   72 +-
 .../PartitionedRegionBucketMgmtHelper.java      |   20 +-
 .../cache/PartitionedRegionDataStore.java       |   24 +-
 .../cache/PartitionedRegionDataView.java        |   21 +-
 .../cache/PartitionedRegionException.java       |   21 +-
 .../internal/cache/PartitionedRegionHelper.java |   24 +-
 .../cache/PartitionedRegionQueryEvaluator.java  |   20 +-
 .../internal/cache/PartitionedRegionStats.java  |   33 +-
 .../internal/cache/PartitionedRegionStatus.java |   21 +-
 .../gemfire/internal/cache/PeerTXStateStub.java |   21 +-
 .../internal/cache/PersistentOplogSet.java      |   21 +-
 .../internal/cache/PlaceHolderDiskRegion.java   |   23 +-
 .../gemfire/internal/cache/PoolFactoryImpl.java |  195 +-
 .../gemfire/internal/cache/PoolManagerImpl.java |   40 +-
 .../gemfire/internal/cache/PoolStats.java       |   21 +-
 .../cache/PreferBytesCachedDeserializable.java  |   21 +-
 .../internal/cache/PrimaryBucketException.java  |   21 +-
 .../cache/ProfileExchangeProcessor.java         |   21 +-
 .../internal/cache/ProxyBucketRegion.java       |   21 +-
 .../gemfire/internal/cache/ProxyRegionMap.java  |   21 +-
 .../cache/PutAllPartialResultException.java     |   21 +-
 .../gemfire/internal/cache/QueuedOperation.java |   21 +-
 .../internal/cache/RegionClearedException.java  |   21 +-
 .../gemfire/internal/cache/RegionEntry.java     |   21 +-
 .../internal/cache/RegionEntryContext.java      |   21 +-
 .../internal/cache/RegionEntryFactory.java      |   21 +-
 .../gemfire/internal/cache/RegionEventImpl.java |   23 +-
 .../internal/cache/RegionEvictorTask.java       |   21 +-
 .../internal/cache/RegionExpiryTask.java        |   24 +-
 .../internal/cache/RegionFactoryImpl.java       |   21 +-
 .../internal/cache/RegionIdleExpiryTask.java    |   26 +-
 .../gemfire/internal/cache/RegionListener.java  |   46 +
 .../gemfire/internal/cache/RegionMap.java       |   21 +-
 .../internal/cache/RegionMapFactory.java        |   21 +-
 .../gemfire/internal/cache/RegionQueue.java     |   21 +-
 .../internal/cache/RegionQueueException.java    |   21 +-
 .../gemfire/internal/cache/RegionStatus.java    |   21 +-
 .../internal/cache/RegionTTLExpiryTask.java     |   23 +-
 .../internal/cache/ReleaseClearLockMessage.java |   23 +-
 .../cache/ReliableDistributionData.java         |   21 +-
 .../internal/cache/ReliableMessageQueue.java    |   21 +-
 .../cache/ReliableMessageQueueFactory.java      |   21 +-
 .../cache/ReliableMessageQueueFactoryImpl.java  |   21 +-
 .../cache/RemoteContainsKeyValueMessage.java    |   20 +-
 .../internal/cache/RemoteDestroyMessage.java    |   21 +-
 .../internal/cache/RemoteFetchEntryMessage.java |   20 +-
 .../cache/RemoteFetchVersionMessage.java        |   20 +-
 .../internal/cache/RemoteGetMessage.java        |   21 +-
 .../internal/cache/RemoteInvalidateMessage.java |   20 +-
 .../cache/RemoteOperationException.java         |   21 +-
 .../internal/cache/RemoteOperationMessage.java  |   23 +-
 .../RemoteOperationMessageWithDirectReply.java  |   21 +-
 .../internal/cache/RemotePutAllMessage.java     |   21 +-
 .../internal/cache/RemotePutMessage.java        |   77 +-
 .../internal/cache/RemoteRegionOperation.java   |   20 +-
 .../internal/cache/RemoteRemoveAllMessage.java  |   21 +-
 .../gemfire/internal/cache/RoleEventImpl.java   |   21 +-
 .../cache/SearchLoadAndWriteProcessor.java      |   23 +-
 .../internal/cache/SendQueueOperation.java      |   21 +-
 .../internal/cache/SerializationHelper.java     |   16 +
 .../internal/cache/ServerPingMessage.java       |   16 +
 .../internal/cache/StateFlushOperation.java     |   30 +-
 .../cache/StoreAllCachedDeserializable.java     |   21 +-
 .../internal/cache/TXBucketRegionState.java     |   21 +-
 .../gemfire/internal/cache/TXCommitMessage.java |  132 +-
 .../gemfire/internal/cache/TXEntry.java         |   21 +-
 .../gemfire/internal/cache/TXEntryState.java    |   40 +-
 .../internal/cache/TXEntryStateFactory.java     |   21 +-
 .../internal/cache/TXEntryUserAttrState.java    |   21 +-
 .../gemfire/internal/cache/TXEvent.java         |   21 +-
 .../internal/cache/TXFarSideCMTracker.java      |   23 +-
 .../gemstone/gemfire/internal/cache/TXId.java   |   22 +-
 .../gemfire/internal/cache/TXLockRequest.java   |   21 +-
 .../gemfire/internal/cache/TXManagerImpl.java   |   34 +-
 .../gemfire/internal/cache/TXMessage.java       |   21 +-
 .../internal/cache/TXRegionLockRequestImpl.java |   21 +-
 .../gemfire/internal/cache/TXRegionState.java   |   71 +-
 .../internal/cache/TXRemoteCommitMessage.java   |   21 +-
 .../internal/cache/TXRemoteRollbackMessage.java |   21 +-
 .../internal/cache/TXReservationMgr.java        |   21 +-
 .../gemfire/internal/cache/TXRmtEvent.java      |   21 +-
 .../gemfire/internal/cache/TXState.java         |   23 +-
 .../internal/cache/TXStateInterface.java        |   21 +-
 .../gemfire/internal/cache/TXStateProxy.java    |   21 +-
 .../internal/cache/TXStateProxyImpl.java        |   25 +-
 .../gemfire/internal/cache/TXStateStub.java     |   21 +-
 .../cache/TXSynchronizationRunnable.java        |   21 +-
 .../cache/TestHeapThresholdObserver.java        |   21 +-
 .../cache/TimestampedEntryEventImpl.java        |   21 +-
 .../gemstone/gemfire/internal/cache/Token.java  |   21 +-
 .../internal/cache/TombstoneService.java        |   24 +-
 .../internal/cache/TransactionMessage.java      |   21 +-
 .../gemfire/internal/cache/TxEntryFactory.java  |   16 +
 .../internal/cache/UnsharedImageState.java      |   21 +-
 .../cache/UpdateAttributesProcessor.java        |   23 +-
 .../cache/UpdateEntryVersionOperation.java      |   22 +-
 .../gemfire/internal/cache/UpdateOperation.java |   20 +-
 .../cache/UserSpecifiedDiskStoreAttributes.java |   21 +-
 .../cache/UserSpecifiedRegionAttributes.java    |   21 +-
 .../internal/cache/VMCachedDeserializable.java  |   21 +-
 .../gemfire/internal/cache/VMLRURegionMap.java  |   21 +-
 .../gemfire/internal/cache/VMRegionMap.java     |   21 +-
 .../cache/VMStatsDiskLRURegionEntry.java        |   21 +-
 .../cache/VMStatsDiskLRURegionEntryHeap.java    |   21 +-
 .../VMStatsDiskLRURegionEntryHeapIntKey.java    |   21 +-
 .../VMStatsDiskLRURegionEntryHeapLongKey.java   |   21 +-
 .../VMStatsDiskLRURegionEntryHeapObjectKey.java |   21 +-
 ...VMStatsDiskLRURegionEntryHeapStringKey1.java |   21 +-
 ...VMStatsDiskLRURegionEntryHeapStringKey2.java |   21 +-
 .../VMStatsDiskLRURegionEntryHeapUUIDKey.java   |   21 +-
 .../cache/VMStatsDiskLRURegionEntryOffHeap.java |   21 +-
 .../VMStatsDiskLRURegionEntryOffHeapIntKey.java |   21 +-
 ...VMStatsDiskLRURegionEntryOffHeapLongKey.java |   21 +-
 ...StatsDiskLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...tatsDiskLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...tatsDiskLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...VMStatsDiskLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../internal/cache/VMStatsDiskRegionEntry.java  |   21 +-
 .../cache/VMStatsDiskRegionEntryHeap.java       |   21 +-
 .../cache/VMStatsDiskRegionEntryHeapIntKey.java |   21 +-
 .../VMStatsDiskRegionEntryHeapLongKey.java      |   21 +-
 .../VMStatsDiskRegionEntryHeapObjectKey.java    |   21 +-
 .../VMStatsDiskRegionEntryHeapStringKey1.java   |   21 +-
 .../VMStatsDiskRegionEntryHeapStringKey2.java   |   21 +-
 .../VMStatsDiskRegionEntryHeapUUIDKey.java      |   21 +-
 .../cache/VMStatsDiskRegionEntryOffHeap.java    |   21 +-
 .../VMStatsDiskRegionEntryOffHeapIntKey.java    |   21 +-
 .../VMStatsDiskRegionEntryOffHeapLongKey.java   |   21 +-
 .../VMStatsDiskRegionEntryOffHeapObjectKey.java |   21 +-
 ...VMStatsDiskRegionEntryOffHeapStringKey1.java |   21 +-
 ...VMStatsDiskRegionEntryOffHeapStringKey2.java |   21 +-
 .../VMStatsDiskRegionEntryOffHeapUUIDKey.java   |   21 +-
 .../internal/cache/VMStatsLRURegionEntry.java   |   21 +-
 .../cache/VMStatsLRURegionEntryHeap.java        |   21 +-
 .../cache/VMStatsLRURegionEntryHeapIntKey.java  |   21 +-
 .../cache/VMStatsLRURegionEntryHeapLongKey.java |   21 +-
 .../VMStatsLRURegionEntryHeapObjectKey.java     |   21 +-
 .../VMStatsLRURegionEntryHeapStringKey1.java    |   21 +-
 .../VMStatsLRURegionEntryHeapStringKey2.java    |   21 +-
 .../cache/VMStatsLRURegionEntryHeapUUIDKey.java |   21 +-
 .../cache/VMStatsLRURegionEntryOffHeap.java     |   21 +-
 .../VMStatsLRURegionEntryOffHeapIntKey.java     |   21 +-
 .../VMStatsLRURegionEntryOffHeapLongKey.java    |   21 +-
 .../VMStatsLRURegionEntryOffHeapObjectKey.java  |   21 +-
 .../VMStatsLRURegionEntryOffHeapStringKey1.java |   21 +-
 .../VMStatsLRURegionEntryOffHeapStringKey2.java |   21 +-
 .../VMStatsLRURegionEntryOffHeapUUIDKey.java    |   21 +-
 .../internal/cache/VMStatsRegionEntry.java      |   21 +-
 .../internal/cache/VMStatsRegionEntryHeap.java  |   21 +-
 .../cache/VMStatsRegionEntryHeapIntKey.java     |   21 +-
 .../cache/VMStatsRegionEntryHeapLongKey.java    |   21 +-
 .../cache/VMStatsRegionEntryHeapObjectKey.java  |   21 +-
 .../cache/VMStatsRegionEntryHeapStringKey1.java |   21 +-
 .../cache/VMStatsRegionEntryHeapStringKey2.java |   21 +-
 .../cache/VMStatsRegionEntryHeapUUIDKey.java    |   21 +-
 .../cache/VMStatsRegionEntryOffHeap.java        |   21 +-
 .../cache/VMStatsRegionEntryOffHeapIntKey.java  |   21 +-
 .../cache/VMStatsRegionEntryOffHeapLongKey.java |   21 +-
 .../VMStatsRegionEntryOffHeapObjectKey.java     |   21 +-
 .../VMStatsRegionEntryOffHeapStringKey1.java    |   21 +-
 .../VMStatsRegionEntryOffHeapStringKey2.java    |   21 +-
 .../cache/VMStatsRegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VMThinDiskLRURegionEntry.java         |   21 +-
 .../cache/VMThinDiskLRURegionEntryHeap.java     |   21 +-
 .../VMThinDiskLRURegionEntryHeapIntKey.java     |   21 +-
 .../VMThinDiskLRURegionEntryHeapLongKey.java    |   21 +-
 .../VMThinDiskLRURegionEntryHeapObjectKey.java  |   21 +-
 .../VMThinDiskLRURegionEntryHeapStringKey1.java |   21 +-
 .../VMThinDiskLRURegionEntryHeapStringKey2.java |   21 +-
 .../VMThinDiskLRURegionEntryHeapUUIDKey.java    |   21 +-
 .../cache/VMThinDiskLRURegionEntryOffHeap.java  |   21 +-
 .../VMThinDiskLRURegionEntryOffHeapIntKey.java  |   21 +-
 .../VMThinDiskLRURegionEntryOffHeapLongKey.java |   21 +-
 ...MThinDiskLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...ThinDiskLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...ThinDiskLRURegionEntryOffHeapStringKey2.java |   21 +-
 .../VMThinDiskLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../internal/cache/VMThinDiskRegionEntry.java   |   21 +-
 .../cache/VMThinDiskRegionEntryHeap.java        |   21 +-
 .../cache/VMThinDiskRegionEntryHeapIntKey.java  |   21 +-
 .../cache/VMThinDiskRegionEntryHeapLongKey.java |   21 +-
 .../VMThinDiskRegionEntryHeapObjectKey.java     |   21 +-
 .../VMThinDiskRegionEntryHeapStringKey1.java    |   21 +-
 .../VMThinDiskRegionEntryHeapStringKey2.java    |   21 +-
 .../cache/VMThinDiskRegionEntryHeapUUIDKey.java |   21 +-
 .../cache/VMThinDiskRegionEntryOffHeap.java     |   21 +-
 .../VMThinDiskRegionEntryOffHeapIntKey.java     |   21 +-
 .../VMThinDiskRegionEntryOffHeapLongKey.java    |   21 +-
 .../VMThinDiskRegionEntryOffHeapObjectKey.java  |   21 +-
 .../VMThinDiskRegionEntryOffHeapStringKey1.java |   21 +-
 .../VMThinDiskRegionEntryOffHeapStringKey2.java |   21 +-
 .../VMThinDiskRegionEntryOffHeapUUIDKey.java    |   21 +-
 .../internal/cache/VMThinLRURegionEntry.java    |   21 +-
 .../cache/VMThinLRURegionEntryHeap.java         |   21 +-
 .../cache/VMThinLRURegionEntryHeapIntKey.java   |   21 +-
 .../cache/VMThinLRURegionEntryHeapLongKey.java  |   21 +-
 .../VMThinLRURegionEntryHeapObjectKey.java      |   21 +-
 .../VMThinLRURegionEntryHeapStringKey1.java     |   21 +-
 .../VMThinLRURegionEntryHeapStringKey2.java     |   21 +-
 .../cache/VMThinLRURegionEntryHeapUUIDKey.java  |   21 +-
 .../cache/VMThinLRURegionEntryOffHeap.java      |   21 +-
 .../VMThinLRURegionEntryOffHeapIntKey.java      |   21 +-
 .../VMThinLRURegionEntryOffHeapLongKey.java     |   21 +-
 .../VMThinLRURegionEntryOffHeapObjectKey.java   |   21 +-
 .../VMThinLRURegionEntryOffHeapStringKey1.java  |   21 +-
 .../VMThinLRURegionEntryOffHeapStringKey2.java  |   21 +-
 .../VMThinLRURegionEntryOffHeapUUIDKey.java     |   21 +-
 .../internal/cache/VMThinRegionEntry.java       |   26 +-
 .../internal/cache/VMThinRegionEntryHeap.java   |   21 +-
 .../cache/VMThinRegionEntryHeapIntKey.java      |   21 +-
 .../cache/VMThinRegionEntryHeapLongKey.java     |   21 +-
 .../cache/VMThinRegionEntryHeapObjectKey.java   |   21 +-
 .../cache/VMThinRegionEntryHeapStringKey1.java  |   21 +-
 .../cache/VMThinRegionEntryHeapStringKey2.java  |   21 +-
 .../cache/VMThinRegionEntryHeapUUIDKey.java     |   21 +-
 .../cache/VMThinRegionEntryOffHeap.java         |   16 +
 .../cache/VMThinRegionEntryOffHeapIntKey.java   |   21 +-
 .../cache/VMThinRegionEntryOffHeapLongKey.java  |   21 +-
 .../VMThinRegionEntryOffHeapObjectKey.java      |   21 +-
 .../VMThinRegionEntryOffHeapStringKey1.java     |   21 +-
 .../VMThinRegionEntryOffHeapStringKey2.java     |   21 +-
 .../cache/VMThinRegionEntryOffHeapUUIDKey.java  |   21 +-
 .../internal/cache/ValidatingDiskRegion.java    |   21 +-
 .../internal/cache/ValueByteWrapper.java        |   21 +-
 .../internal/cache/VersionTimestamp.java        |   21 +-
 .../cache/VersionedStatsDiskLRURegionEntry.java |   21 +-
 .../VersionedStatsDiskLRURegionEntryHeap.java   |   21 +-
 ...sionedStatsDiskLRURegionEntryHeapIntKey.java |   21 +-
 ...ionedStatsDiskLRURegionEntryHeapLongKey.java |   21 +-
 ...nedStatsDiskLRURegionEntryHeapObjectKey.java |   21 +-
 ...edStatsDiskLRURegionEntryHeapStringKey1.java |   21 +-
 ...edStatsDiskLRURegionEntryHeapStringKey2.java |   21 +-
 ...ionedStatsDiskLRURegionEntryHeapUUIDKey.java |   21 +-
 ...VersionedStatsDiskLRURegionEntryOffHeap.java |   21 +-
 ...nedStatsDiskLRURegionEntryOffHeapIntKey.java |   21 +-
 ...edStatsDiskLRURegionEntryOffHeapLongKey.java |   21 +-
 ...StatsDiskLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...tatsDiskLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...tatsDiskLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...edStatsDiskLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedStatsDiskRegionEntry.java    |   21 +-
 .../VersionedStatsDiskRegionEntryHeap.java      |   21 +-
 ...VersionedStatsDiskRegionEntryHeapIntKey.java |   21 +-
 ...ersionedStatsDiskRegionEntryHeapLongKey.java |   21 +-
 ...sionedStatsDiskRegionEntryHeapObjectKey.java |   21 +-
 ...ionedStatsDiskRegionEntryHeapStringKey1.java |   21 +-
 ...ionedStatsDiskRegionEntryHeapStringKey2.java |   21 +-
 ...ersionedStatsDiskRegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedStatsDiskRegionEntryOffHeap.java   |   21 +-
 ...sionedStatsDiskRegionEntryOffHeapIntKey.java |   21 +-
 ...ionedStatsDiskRegionEntryOffHeapLongKey.java |   21 +-
 ...nedStatsDiskRegionEntryOffHeapObjectKey.java |   21 +-
 ...edStatsDiskRegionEntryOffHeapStringKey1.java |   21 +-
 ...edStatsDiskRegionEntryOffHeapStringKey2.java |   21 +-
 ...ionedStatsDiskRegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedStatsLRURegionEntry.java     |   21 +-
 .../cache/VersionedStatsLRURegionEntryHeap.java |   21 +-
 .../VersionedStatsLRURegionEntryHeapIntKey.java |   21 +-
 ...VersionedStatsLRURegionEntryHeapLongKey.java |   21 +-
 ...rsionedStatsLRURegionEntryHeapObjectKey.java |   21 +-
 ...sionedStatsLRURegionEntryHeapStringKey1.java |   21 +-
 ...sionedStatsLRURegionEntryHeapStringKey2.java |   21 +-
 ...VersionedStatsLRURegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedStatsLRURegionEntryOffHeap.java    |   21 +-
 ...rsionedStatsLRURegionEntryOffHeapIntKey.java |   21 +-
 ...sionedStatsLRURegionEntryOffHeapLongKey.java |   21 +-
 ...onedStatsLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...nedStatsLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...nedStatsLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...sionedStatsLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedStatsRegionEntry.java        |   21 +-
 .../cache/VersionedStatsRegionEntryHeap.java    |   21 +-
 .../VersionedStatsRegionEntryHeapIntKey.java    |   21 +-
 .../VersionedStatsRegionEntryHeapLongKey.java   |   21 +-
 .../VersionedStatsRegionEntryHeapObjectKey.java |   21 +-
 ...VersionedStatsRegionEntryHeapStringKey1.java |   21 +-
 ...VersionedStatsRegionEntryHeapStringKey2.java |   21 +-
 .../VersionedStatsRegionEntryHeapUUIDKey.java   |   21 +-
 .../cache/VersionedStatsRegionEntryOffHeap.java |   21 +-
 .../VersionedStatsRegionEntryOffHeapIntKey.java |   21 +-
 ...VersionedStatsRegionEntryOffHeapLongKey.java |   21 +-
 ...rsionedStatsRegionEntryOffHeapObjectKey.java |   21 +-
 ...sionedStatsRegionEntryOffHeapStringKey1.java |   21 +-
 ...sionedStatsRegionEntryOffHeapStringKey2.java |   21 +-
 ...VersionedStatsRegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedThinDiskLRURegionEntry.java  |   21 +-
 .../VersionedThinDiskLRURegionEntryHeap.java    |   21 +-
 ...rsionedThinDiskLRURegionEntryHeapIntKey.java |   21 +-
 ...sionedThinDiskLRURegionEntryHeapLongKey.java |   21 +-
 ...onedThinDiskLRURegionEntryHeapObjectKey.java |   21 +-
 ...nedThinDiskLRURegionEntryHeapStringKey1.java |   21 +-
 ...nedThinDiskLRURegionEntryHeapStringKey2.java |   21 +-
 ...sionedThinDiskLRURegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedThinDiskLRURegionEntryOffHeap.java |   21 +-
 ...onedThinDiskLRURegionEntryOffHeapIntKey.java |   21 +-
 ...nedThinDiskLRURegionEntryOffHeapLongKey.java |   21 +-
 ...dThinDiskLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...ThinDiskLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...ThinDiskLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...nedThinDiskLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedThinDiskRegionEntry.java     |   21 +-
 .../cache/VersionedThinDiskRegionEntryHeap.java |   21 +-
 .../VersionedThinDiskRegionEntryHeapIntKey.java |   21 +-
 ...VersionedThinDiskRegionEntryHeapLongKey.java |   21 +-
 ...rsionedThinDiskRegionEntryHeapObjectKey.java |   21 +-
 ...sionedThinDiskRegionEntryHeapStringKey1.java |   21 +-
 ...sionedThinDiskRegionEntryHeapStringKey2.java |   21 +-
 ...VersionedThinDiskRegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedThinDiskRegionEntryOffHeap.java    |   21 +-
 ...rsionedThinDiskRegionEntryOffHeapIntKey.java |   21 +-
 ...sionedThinDiskRegionEntryOffHeapLongKey.java |   21 +-
 ...onedThinDiskRegionEntryOffHeapObjectKey.java |   21 +-
 ...nedThinDiskRegionEntryOffHeapStringKey1.java |   21 +-
 ...nedThinDiskRegionEntryOffHeapStringKey2.java |   21 +-
 ...sionedThinDiskRegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedThinLRURegionEntry.java      |   21 +-
 .../cache/VersionedThinLRURegionEntryHeap.java  |   21 +-
 .../VersionedThinLRURegionEntryHeapIntKey.java  |   21 +-
 .../VersionedThinLRURegionEntryHeapLongKey.java |   21 +-
 ...ersionedThinLRURegionEntryHeapObjectKey.java |   21 +-
 ...rsionedThinLRURegionEntryHeapStringKey1.java |   21 +-
 ...rsionedThinLRURegionEntryHeapStringKey2.java |   21 +-
 .../VersionedThinLRURegionEntryHeapUUIDKey.java |   21 +-
 .../VersionedThinLRURegionEntryOffHeap.java     |   21 +-
 ...ersionedThinLRURegionEntryOffHeapIntKey.java |   21 +-
 ...rsionedThinLRURegionEntryOffHeapLongKey.java |   21 +-
 ...ionedThinLRURegionEntryOffHeapObjectKey.java |   21 +-
 ...onedThinLRURegionEntryOffHeapStringKey1.java |   21 +-
 ...onedThinLRURegionEntryOffHeapStringKey2.java |   21 +-
 ...rsionedThinLRURegionEntryOffHeapUUIDKey.java |   21 +-
 .../cache/VersionedThinRegionEntry.java         |   21 +-
 .../cache/VersionedThinRegionEntryHeap.java     |   21 +-
 .../VersionedThinRegionEntryHeapIntKey.java     |   21 +-
 .../VersionedThinRegionEntryHeapLongKey.java    |   21 +-
 .../VersionedThinRegionEntryHeapObjectKey.java  |   21 +-
 .../VersionedThinRegionEntryHeapStringKey1.java |   21 +-
 .../VersionedThinRegionEntryHeapStringKey2.java |   21 +-
 .../VersionedThinRegionEntryHeapUUIDKey.java    |   21 +-
 .../cache/VersionedThinRegionEntryOffHeap.java  |   21 +-
 .../VersionedThinRegionEntryOffHeapIntKey.java  |   21 +-
 .../VersionedThinRegionEntryOffHeapLongKey.java |   21 +-
 ...ersionedThinRegionEntryOffHeapObjectKey.java |   21 +-
 ...rsionedThinRegionEntryOffHeapStringKey1.java |   21 +-
 ...rsionedThinRegionEntryOffHeapStringKey2.java |   21 +-
 .../VersionedThinRegionEntryOffHeapUUIDKey.java |   21 +-
 .../internal/cache/WrappedCallbackArgument.java |   21 +-
 .../cache/WrappedRegionMembershipListener.java  |   21 +-
 .../CompressedCachedDeserializable.java         |   23 +-
 .../SnappyCompressedCachedDeserializable.java   |   23 +-
 .../internal/cache/control/FilterByPath.java    |   21 +-
 .../cache/control/HeapMemoryMonitor.java        |   35 +-
 .../cache/control/InternalResourceManager.java  |   49 +-
 .../internal/cache/control/MemoryEvent.java     |   29 +-
 .../cache/control/MemoryThresholds.java         |   16 +
 .../cache/control/OffHeapMemoryMonitor.java     |  256 +-
 .../control/PartitionRebalanceDetailsImpl.java  |   29 +-
 .../cache/control/RebalanceOperationImpl.java   |  178 +-
 .../cache/control/RebalanceResultsImpl.java     |   37 +-
 .../internal/cache/control/RegionFilter.java    |   21 +-
 .../internal/cache/control/ResourceAdvisor.java |   21 +-
 .../internal/cache/control/ResourceEvent.java   |   20 +-
 .../cache/control/ResourceListener.java         |   21 +-
 .../cache/control/ResourceManagerStats.java     |   34 +-
 .../internal/cache/control/ResourceMonitor.java |   16 +
 .../gemfire/internal/cache/delta/Delta.java     |   21 +-
 .../internal/cache/doc-files/properties.html    |   42 +-
 .../cache/execute/AbstractExecution.java        |   21 +-
 .../cache/execute/BucketMovedException.java     |   21 +-
 .../cache/execute/DefaultResultCollector.java   |   20 +-
 .../DistributedRegionFunctionExecutor.java      |   20 +-
 .../DistributedRegionFunctionResultSender.java  |   21 +-
 .../DistributedRegionFunctionResultWaiter.java  |   21 +-
 .../cache/execute/FunctionContextImpl.java      |   21 +-
 .../execute/FunctionExecutionNodePruner.java    |   21 +-
 .../cache/execute/FunctionRemoteContext.java    |   21 +-
 .../cache/execute/FunctionServiceStats.java     |   24 +-
 .../internal/cache/execute/FunctionStats.java   |   21 +-
 .../FunctionStreamingResultCollector.java       |   21 +-
 .../cache/execute/InternalExecution.java        |   21 +-
 .../execute/InternalFunctionException.java      |   21 +-
 ...ternalFunctionInvocationTargetException.java |   21 +-
 .../cache/execute/InternalFunctionService.java  |   24 +-
 .../execute/InternalRegionFunctionContext.java  |   21 +-
 .../cache/execute/InternalResultSender.java     |   21 +-
 .../cache/execute/LocalResultCollector.java     |   20 +-
 .../cache/execute/LocalResultCollectorImpl.java |   21 +-
 .../cache/execute/MemberFunctionExecutor.java   |   21 +-
 .../execute/MemberFunctionResultSender.java     |   20 +-
 .../execute/MemberFunctionResultWaiter.java     |   21 +-
 .../cache/execute/MemberMappedArgument.java     |   20 +-
 .../execute/MultiRegionFunctionContext.java     |   21 +-
 .../execute/MultiRegionFunctionContextImpl.java |   21 +-
 .../execute/MultiRegionFunctionExecutor.java    |   21 +-
 .../MultiRegionFunctionResultWaiter.java        |   21 +-
 .../internal/cache/execute/NoResult.java        |   20 +-
 .../PartitionedRegionFunctionExecutor.java      |   21 +-
 .../PartitionedRegionFunctionResultSender.java  |   20 +-
 .../PartitionedRegionFunctionResultWaiter.java  |   21 +-
 .../execute/RegionFunctionContextImpl.java      |   21 +-
 .../cache/execute/ServerFunctionExecutor.java   |   39 +-
 .../execute/ServerRegionFunctionExecutor.java   |   23 +-
 .../ServerToClientFunctionResultSender.java     |   20 +-
 .../ServerToClientFunctionResultSender65.java   |   20 +-
 .../execute/StreamingFunctionOperation.java     |   21 +-
 .../execute/TransactionFunctionService.java     |  184 -
 .../cache/execute/util/CommitFunction.java      |   22 +-
 .../util/FindRestEnabledServersFunction.java    |   23 +-
 .../execute/util/NestedTransactionFunction.java |   21 +-
 .../cache/execute/util/RollbackFunction.java    |   22 +-
 .../internal/cache/extension/Extensible.java    |   21 +-
 .../internal/cache/extension/Extension.java     |   21 +-
 .../cache/extension/ExtensionPoint.java         |   21 +-
 .../cache/extension/SimpleExtensionPoint.java   |   23 +-
 .../internal/cache/ha/HAContainerMap.java       |   21 +-
 .../internal/cache/ha/HAContainerRegion.java    |   21 +-
 .../internal/cache/ha/HAContainerWrapper.java   |   21 +-
 .../internal/cache/ha/HARegionQueue.java        |   28 +-
 .../cache/ha/HARegionQueueAttributes.java       |   21 +-
 .../internal/cache/ha/HARegionQueueStats.java   |   21 +-
 .../internal/cache/ha/QueueRemovalMessage.java  |   21 +-
 .../internal/cache/ha/ThreadIdentifier.java     |   21 +-
 .../locks/GFEAbstractQueuedSynchronizer.java    |   21 +-
 .../locks/ReentrantReadWriteWriteShareLock.java |   21 +-
 .../cache/locks/TXLessorDepartureHandler.java   |   21 +-
 .../internal/cache/locks/TXLockBatch.java       |   21 +-
 .../gemfire/internal/cache/locks/TXLockId.java  |   21 +-
 .../internal/cache/locks/TXLockIdImpl.java      |   21 +-
 .../internal/cache/locks/TXLockService.java     |   21 +-
 .../internal/cache/locks/TXLockServiceImpl.java |   21 +-
 .../internal/cache/locks/TXLockToken.java       |   21 +-
 .../locks/TXLockUpdateParticipantsMessage.java  |   21 +-
 .../locks/TXOriginatorRecoveryProcessor.java    |   21 +-
 .../locks/TXRecoverGrantorMessageProcessor.java |   21 +-
 .../cache/locks/TXRegionLockRequest.java        |   21 +-
 .../gemfire/internal/cache/lru/EnableLRU.java   |   21 +-
 .../gemfire/internal/cache/lru/HeapEvictor.java |   21 +-
 .../cache/lru/HeapLRUCapacityController.java    |   21 +-
 .../internal/cache/lru/HeapLRUStatistics.java   |   21 +-
 .../internal/cache/lru/LRUAlgorithm.java        |   21 +-
 .../cache/lru/LRUCapacityController.java        |   21 +-
 .../internal/cache/lru/LRUClockNode.java        |   21 +-
 .../gemfire/internal/cache/lru/LRUEntry.java    |   21 +-
 .../internal/cache/lru/LRUMapCallbacks.java     |   21 +-
 .../internal/cache/lru/LRUStatistics.java       |   21 +-
 .../cache/lru/MemLRUCapacityController.java     |   21 +-
 .../internal/cache/lru/NewLIFOClockHand.java    |   99 +-
 .../internal/cache/lru/NewLRUClockHand.java     |   35 +-
 .../internal/cache/lru/OffHeapEvictor.java      |   21 +-
 .../gemfire/internal/cache/lru/Sizeable.java    |   21 +-
 .../operations/ContainsKeyOperationContext.java |   21 +-
 .../gemfire/internal/cache/package.html         |   16 +
 .../AllBucketProfilesUpdateMessage.java         |   21 +-
 .../partitioned/BecomePrimaryBucketMessage.java |   21 +-
 .../internal/cache/partitioned/Bucket.java      |   21 +-
 .../cache/partitioned/BucketBackupMessage.java  |   20 +-
 .../cache/partitioned/BucketCountLoadProbe.java |   75 +
 .../partitioned/BucketProfileUpdateMessage.java |   23 +-
 .../cache/partitioned/BucketSizeMessage.java    |   21 +-
 .../partitioned/ContainsKeyValueMessage.java    |   20 +-
 .../cache/partitioned/CreateBucketMessage.java  |   21 +-
 .../partitioned/CreateMissingBucketsTask.java   |   23 +-
 .../partitioned/DeposePrimaryBucketMessage.java |   21 +-
 .../cache/partitioned/DestroyMessage.java       |   21 +-
 .../DestroyRegionOnDataStoreMessage.java        |   20 +-
 .../partitioned/DumpAllPRConfigMessage.java     |   20 +-
 .../cache/partitioned/DumpB2NRegion.java        |   20 +-
 .../cache/partitioned/DumpBucketsMessage.java   |   21 +-
 .../partitioned/EndBucketCreationMessage.java   |   21 +-
 .../partitioned/FetchBulkEntriesMessage.java    |   21 +-
 .../cache/partitioned/FetchEntriesMessage.java  |   32 +-
 .../cache/partitioned/FetchEntryMessage.java    |   20 +-
 .../cache/partitioned/FetchKeysMessage.java     |   21 +-
 .../FetchPartitionDetailsMessage.java           |   21 +-
 .../cache/partitioned/FlushMessage.java         |   21 +-
 .../internal/cache/partitioned/GetMessage.java  |   27 +-
 .../partitioned/IdentityRequestMessage.java     |   20 +-
 .../partitioned/IdentityUpdateMessage.java      |   21 +-
 .../cache/partitioned/IndexCreationMsg.java     |   21 +-
 .../cache/partitioned/InterestEventMessage.java |   21 +-
 .../cache/partitioned/InternalPRInfo.java       |   21 +-
 .../partitioned/InternalPartitionDetails.java   |   21 +-
 .../cache/partitioned/InvalidateMessage.java    |   20 +-
 .../internal/cache/partitioned/LoadProbe.java   |   21 +-
 .../internal/cache/partitioned/LockObject.java  |   21 +-
 .../partitioned/ManageBackupBucketMessage.java  |   21 +-
 .../cache/partitioned/ManageBucketMessage.java  |   21 +-
 .../cache/partitioned/MoveBucketMessage.java    |   21 +-
 .../cache/partitioned/OfflineMemberDetails.java |   23 +-
 .../partitioned/OfflineMemberDetailsImpl.java   |   21 +-
 .../cache/partitioned/PREntriesIterator.java    |   21 +-
 .../PRFunctionStreamingResultCollector.java     |   21 +-
 .../internal/cache/partitioned/PRLoad.java      |   21 +-
 .../PRLocallyDestroyedException.java            |   21 +-
 .../cache/partitioned/PRSanityCheckMessage.java |   21 +-
 .../cache/partitioned/PRTombstoneMessage.java   |   24 +-
 .../PRUpdateEntryVersionMessage.java            |   22 +-
 .../partitioned/PartitionMemberInfoImpl.java    |   21 +-
 .../cache/partitioned/PartitionMessage.java     |   25 +-
 .../PartitionMessageWithDirectReply.java        |   21 +-
 .../partitioned/PartitionRegionInfoImpl.java    |   21 +-
 ...rtitionedRegionFunctionStreamingMessage.java |   21 +-
 .../partitioned/PartitionedRegionObserver.java  |   21 +-
 .../PartitionedRegionObserverAdapter.java       |   21 +-
 .../PartitionedRegionObserverHolder.java        |   23 +-
 .../PartitionedRegionRebalanceOp.java           |  175 +-
 .../partitioned/PrimaryRequestMessage.java      |   21 +-
 .../cache/partitioned/PutAllPRMessage.java      |   58 +-
 .../internal/cache/partitioned/PutMessage.java  |   42 +-
 .../cache/partitioned/QueryMessage.java         |   21 +-
 .../cache/partitioned/RecoveryRunnable.java     |   23 +-
 .../RedundancyAlreadyMetException.java          |   21 +-
 .../cache/partitioned/RedundancyLogger.java     |   21 +-
 .../cache/partitioned/RegionAdvisor.java        |   20 +-
 .../partitioned/RemoteFetchKeysMessage.java     |   21 +-
 .../cache/partitioned/RemoteSizeMessage.java    |   21 +-
 .../cache/partitioned/RemoveAllPRMessage.java   |   21 +-
 .../cache/partitioned/RemoveBucketMessage.java  |   21 +-
 .../cache/partitioned/RemoveIndexesMessage.java |   21 +-
 .../internal/cache/partitioned/SizeMessage.java |   21 +-
 .../cache/partitioned/SizedBasedLoadProbe.java  |   21 +-
 .../StreamingPartitionOperation.java            |   24 +-
 .../partitioned/rebalance/BucketOperator.java   |   67 +-
 .../rebalance/CompositeDirector.java            |   21 +-
 .../rebalance/ExplicitMoveDirector.java         |   21 +-
 .../partitioned/rebalance/FPRDirector.java      |   21 +-
 .../partitioned/rebalance/MoveBuckets.java      |   21 +-
 .../partitioned/rebalance/MovePrimaries.java    |   21 +-
 .../partitioned/rebalance/MovePrimariesFPR.java |   21 +-
 .../rebalance/ParallelBucketOperator.java       |  167 +
 .../rebalance/PartitionedRegionLoadModel.java   |   66 +-
 .../rebalance/PercentageMoveDirector.java       |   21 +-
 .../rebalance/RebalanceDirector.java            |   21 +-
 .../rebalance/RebalanceDirectorAdapter.java     |   21 +-
 .../rebalance/RemoveOverRedundancy.java         |   21 +-
 .../rebalance/SatisfyRedundancy.java            |   28 +-
 .../rebalance/SatisfyRedundancyFPR.java         |   22 +-
 .../rebalance/SimulatedBucketOperator.java      |   32 +-
 .../cache/persistence/BackupInspector.java      |   21 +-
 .../cache/persistence/BackupManager.java        |   23 +-
 .../cache/persistence/BytesAndBits.java         |   21 +-
 .../cache/persistence/CanonicalIdHolder.java    |   21 +-
 .../CreatePersistentRegionProcessor.java        |   21 +-
 .../cache/persistence/DiskExceptionHandler.java |   21 +-
 .../persistence/DiskInitFileInterpreter.java    |   21 +-
 .../cache/persistence/DiskInitFileParser.java   |   21 +-
 .../cache/persistence/DiskRecoveryStore.java    |   23 +-
 .../cache/persistence/DiskRegionView.java       |   21 +-
 .../cache/persistence/DiskStoreFilter.java      |   23 +-
 .../internal/cache/persistence/DiskStoreID.java |   21 +-
 .../persistence/MembershipFlushRequest.java     |   21 +-
 .../persistence/MembershipViewRequest.java      |   21 +-
 .../internal/cache/persistence/OplogType.java   |   18 +-
 .../cache/persistence/PRPersistentConfig.java   |   21 +-
 .../cache/persistence/PersistenceAdvisor.java   |   21 +-
 .../persistence/PersistenceAdvisorImpl.java     |   23 +-
 .../persistence/PersistenceObserverHolder.java  |   21 +-
 .../cache/persistence/PersistentMemberID.java   |   21 +-
 .../persistence/PersistentMemberManager.java    |   23 +-
 .../persistence/PersistentMemberPattern.java    |   23 +-
 .../persistence/PersistentMemberState.java      |   21 +-
 .../cache/persistence/PersistentMemberView.java |   21 +-
 .../persistence/PersistentMembershipView.java   |   23 +-
 .../persistence/PersistentStateListener.java    |   23 +-
 .../PersistentStateQueryMessage.java            |   21 +-
 .../PersistentStateQueryResults.java            |   21 +-
 .../PrepareNewPersistentMemberMessage.java      |   21 +-
 .../RemovePersistentMemberMessage.java          |   21 +-
 .../cache/persistence/RestoreScript.java        |   21 +-
 .../persistence/UninterruptibleFileChannel.java |   18 +-
 .../UninterruptibleRandomAccessFile.java        |   16 +
 .../persistence/query/CloseableIterator.java    |   18 +-
 .../persistence/query/IdentityExtractor.java    |   16 +
 .../cache/persistence/query/IndexMap.java       |   21 +-
 .../cache/persistence/query/ResultBag.java      |   21 +-
 .../cache/persistence/query/ResultList.java     |   21 +-
 .../cache/persistence/query/ResultMap.java      |   21 +-
 .../cache/persistence/query/ResultSet.java      |   21 +-
 .../persistence/query/SortKeyExtractor.java     |   16 +
 .../query/TemporaryResultSetFactory.java        |   21 +-
 .../persistence/query/mock/ByteComparator.java  |   21 +-
 .../mock/CachedDeserializableComparator.java    |   23 +-
 .../persistence/query/mock/IndexMapImpl.java    |   21 +-
 .../persistence/query/mock/ItrAdapter.java      |   23 +-
 .../query/mock/NaturalComparator.java           |   16 +
 .../cache/persistence/query/mock/Pair.java      |   21 +-
 .../persistence/query/mock/PairComparator.java  |   21 +-
 .../persistence/query/mock/ResultListImpl.java  |   21 +-
 .../query/mock/ReverseComparator.java           |   21 +-
 .../query/mock/SortedResultBagImpl.java         |   21 +-
 .../query/mock/SortedResultMapImpl.java         |   21 +-
 .../query/mock/SortedResultSetImpl.java         |   21 +-
 .../persistence/soplog/AbstractCompactor.java   |  524 --
 .../soplog/AbstractKeyValueIterator.java        |   67 -
 .../soplog/AbstractSortedReader.java            |  126 -
 .../soplog/ArraySerializedComparator.java       |  135 -
 .../persistence/soplog/ByteComparator.java      |   21 +-
 .../cache/persistence/soplog/Compactor.java     |  165 -
 .../soplog/CompositeSerializedComparator.java   |   48 -
 .../persistence/soplog/CursorIterator.java      |   21 +-
 .../soplog/DelegatingSerializedComparator.java  |   21 +-
 .../soplog/HFileStoreStatistics.java            |   21 +-
 .../soplog/IndexSerializedComparator.java       |  118 -
 .../persistence/soplog/KeyValueIterator.java    |   21 +-
 .../cache/persistence/soplog/LevelTracker.java  |  111 -
 .../soplog/LexicographicalComparator.java       |  451 --
 .../cache/persistence/soplog/NonCompactor.java  |  101 -
 .../soplog/ReversingSerializedComparator.java   |   58 -
 .../persistence/soplog/SizeTieredCompactor.java |  189 -
 .../cache/persistence/soplog/SoplogToken.java   |  107 -
 .../cache/persistence/soplog/SortedBuffer.java  |  358 --
 .../cache/persistence/soplog/SortedOplog.java   |  149 -
 .../persistence/soplog/SortedOplogFactory.java  |  269 -
 .../persistence/soplog/SortedOplogSet.java      |  109 -
 .../persistence/soplog/SortedOplogSetImpl.java  |  771 ---
 .../soplog/SortedOplogStatistics.java           |   21 +-
 .../cache/persistence/soplog/SortedReader.java  |   21 +-
 .../persistence/soplog/TrackedReference.java    |   21 +-
 .../soplog/hfile/BlockCacheHolder.java          |   30 -
 .../soplog/hfile/HFileSortedOplog.java          |  685 ---
 .../soplog/hfile/HFileSortedOplogFactory.java   |   71 -
 .../soplog/nofile/NoFileSortedOplog.java        |  235 -
 .../soplog/nofile/NoFileSortedOplogFactory.java |   32 -
 .../snapshot/CacheSnapshotServiceImpl.java      |   21 +-
 .../internal/cache/snapshot/ClientExporter.java |   21 +-
 .../cache/snapshot/ExportedRegistry.java        |   21 +-
 .../internal/cache/snapshot/FlowController.java |   21 +-
 .../internal/cache/snapshot/GFSnapshot.java     |   21 +-
 .../internal/cache/snapshot/LocalExporter.java  |   21 +-
 .../snapshot/RegionSnapshotServiceImpl.java     |   21 +-
 .../cache/snapshot/SnapshotFileMapper.java      |   21 +-
 .../cache/snapshot/SnapshotOptionsImpl.java     |   21 +-
 .../internal/cache/snapshot/SnapshotPacket.java |   21 +-
 .../cache/snapshot/WindowedExporter.java        |   21 +-
 .../gemfire/internal/cache/tier/Acceptor.java   |   21 +-
 .../internal/cache/tier/BatchException.java     |   21 +-
 .../internal/cache/tier/CachedRegionHelper.java |   21 +-
 .../internal/cache/tier/ClientHandShake.java    |   21 +-
 .../gemfire/internal/cache/tier/Command.java    |   21 +-
 .../internal/cache/tier/ConnectionProxy.java    |  181 +-
 .../internal/cache/tier/InterestType.java       |   21 +-
 .../cache/tier/InternalBridgeMembership.java    |  715 ---
 .../cache/tier/InternalClientMembership.java    |  626 +++
 .../internal/cache/tier/MessageType.java        |   21 +-
 .../gemfire/internal/cache/tier/package.html    |   16 +
 .../cache/tier/sockets/AcceptorImpl.java        |   81 +-
 .../cache/tier/sockets/BaseCommand.java         |   40 +-
 .../cache/tier/sockets/BaseCommandQuery.java    |   16 +
 .../cache/tier/sockets/CacheClientNotifier.java |   55 +-
 .../tier/sockets/CacheClientNotifierStats.java  |   21 +-
 .../cache/tier/sockets/CacheClientProxy.java    |   60 +-
 .../tier/sockets/CacheClientProxyStats.java     |   21 +-
 .../cache/tier/sockets/CacheClientUpdater.java  |   43 +-
 .../cache/tier/sockets/CacheServerHelper.java   |   21 +-
 .../cache/tier/sockets/CacheServerStats.java    |   21 +-
 .../cache/tier/sockets/ChunkedMessage.java      |   21 +-
 .../tier/sockets/ClientBlacklistProcessor.java  |   25 +-
 .../sockets/ClientDataSerializerMessage.java    |   21 +-
 .../cache/tier/sockets/ClientHealthMonitor.java |   21 +-
 .../tier/sockets/ClientInstantiatorMessage.java |   20 +-
 .../tier/sockets/ClientInterestMessageImpl.java |   21 +-
 .../tier/sockets/ClientMarkerMessageImpl.java   |   21 +-
 .../cache/tier/sockets/ClientMessage.java       |   21 +-
 .../tier/sockets/ClientPingMessageImpl.java     |   21 +-
 .../tier/sockets/ClientProxyMembershipID.java   |   37 +-
 .../tier/sockets/ClientTombstoneMessage.java    |   21 +-
 .../cache/tier/sockets/ClientUpdateMessage.java |   21 +-
 .../tier/sockets/ClientUpdateMessageImpl.java   |   21 +-
 .../cache/tier/sockets/ClientUserAuths.java     |   21 +-
 .../cache/tier/sockets/CommandInitializer.java  |   25 +-
 .../cache/tier/sockets/ConnectionListener.java  |   21 +-
 .../tier/sockets/ConnectionListenerAdapter.java |   21 +-
 .../cache/tier/sockets/HAEventWrapper.java      |   21 +-
 .../internal/cache/tier/sockets/HandShake.java  |   43 +-
 .../tier/sockets/InterestResultPolicyImpl.java  |   21 +-
 .../internal/cache/tier/sockets/Message.java    |   44 +-
 .../cache/tier/sockets/MessageStats.java        |   21 +-
 .../cache/tier/sockets/ObjectPartList.java      |   21 +-
 .../cache/tier/sockets/ObjectPartList651.java   |   21 +-
 .../internal/cache/tier/sockets/Part.java       |   25 +-
 .../RemoveClientFromBlacklistMessage.java       |   27 +-
 .../tier/sockets/SerializedObjectPartList.java  |   21 +-
 .../cache/tier/sockets/ServerConnection.java    |   31 +-
 .../tier/sockets/ServerHandShakeProcessor.java  |   21 +-
 .../cache/tier/sockets/ServerQueueStatus.java   |   21 +-
 .../tier/sockets/ServerResponseMatrix.java      |   20 +-
 .../tier/sockets/UnregisterAllInterest.java     |   21 +-
 .../cache/tier/sockets/UserAuthAttributes.java  |   21 +-
 .../cache/tier/sockets/VersionedObjectList.java |   21 +-
 .../cache/tier/sockets/command/AddPdxEnum.java  |   21 +-
 .../cache/tier/sockets/command/AddPdxType.java  |   21 +-
 .../cache/tier/sockets/command/ClearRegion.java |   21 +-
 .../cache/tier/sockets/command/ClientReady.java |   21 +-
 .../tier/sockets/command/CloseConnection.java   |   21 +-
 .../tier/sockets/command/CommitCommand.java     |   25 +-
 .../cache/tier/sockets/command/ContainsKey.java |   21 +-
 .../tier/sockets/command/ContainsKey66.java     |   21 +-
 .../tier/sockets/command/CreateRegion.java      |   21 +-
 .../cache/tier/sockets/command/Default.java     |   21 +-
 .../cache/tier/sockets/command/Destroy.java     |   21 +-
 .../cache/tier/sockets/command/Destroy65.java   |   21 +-
 .../cache/tier/sockets/command/Destroy70.java   |   23 +-
 .../tier/sockets/command/DestroyRegion.java     |   21 +-
 .../tier/sockets/command/ExecuteFunction.java   |   21 +-
 .../tier/sockets/command/ExecuteFunction65.java |   20 +-
 .../tier/sockets/command/ExecuteFunction66.java |   22 +-
 .../tier/sockets/command/ExecuteFunction70.java |   21 +-
 .../sockets/command/ExecuteRegionFunction.java  |   20 +-
 .../command/ExecuteRegionFunction61.java        |   20 +-
 .../command/ExecuteRegionFunction65.java        |   20 +-
 .../command/ExecuteRegionFunction66.java        |   22 +-
 .../command/ExecuteRegionFunctionSingleHop.java |   23 +-
 .../sockets/command/GatewayReceiverCommand.java |   47 +-
 .../cache/tier/sockets/command/Get70.java       |   23 +-
 .../cache/tier/sockets/command/GetAll.java      |   21 +-
 .../cache/tier/sockets/command/GetAll651.java   |   21 +-
 .../cache/tier/sockets/command/GetAll70.java    |   23 +-
 .../cache/tier/sockets/command/GetAllForRI.java |   21 +-
 .../sockets/command/GetAllWithCallback.java     |   21 +-
 .../command/GetClientPRMetadataCommand.java     |   20 +-
 .../command/GetClientPRMetadataCommand66.java   |   20 +-
 .../GetClientPartitionAttributesCommand.java    |   20 +-
 .../GetClientPartitionAttributesCommand66.java  |   20 +-
 .../cache/tier/sockets/command/GetEntry70.java  |   23 +-
 .../tier/sockets/command/GetEntryCommand.java   |   21 +-
 .../sockets/command/GetFunctionAttribute.java   |   21 +-
 .../tier/sockets/command/GetPDXEnumById.java    |   21 +-
 .../tier/sockets/command/GetPDXIdForEnum.java   |   21 +-
 .../tier/sockets/command/GetPDXIdForType.java   |   21 +-
 .../tier/sockets/command/GetPDXTypeById.java    |   21 +-
 .../tier/sockets/command/GetPdxEnums70.java     |   21 +-
 .../tier/sockets/command/GetPdxTypes70.java     |   21 +-
 .../cache/tier/sockets/command/Invalid.java     |   21 +-
 .../cache/tier/sockets/command/Invalidate.java  |   21 +-
 .../tier/sockets/command/Invalidate70.java      |   23 +-
 .../cache/tier/sockets/command/KeySet.java      |   21 +-
 .../cache/tier/sockets/command/MakePrimary.java |   21 +-
 .../tier/sockets/command/ManagementCommand.java |   21 +-
 .../cache/tier/sockets/command/PeriodicAck.java |   21 +-
 .../cache/tier/sockets/command/Ping.java        |   36 +-
 .../cache/tier/sockets/command/Put.java         |   23 +-
 .../cache/tier/sockets/command/Put61.java       |   21 +-
 .../cache/tier/sockets/command/Put65.java       |   21 +-
 .../cache/tier/sockets/command/Put70.java       |   21 +-
 .../cache/tier/sockets/command/PutAll.java      |   21 +-
 .../cache/tier/sockets/command/PutAll70.java    |   21 +-
 .../cache/tier/sockets/command/PutAll80.java    |   21 +-
 .../sockets/command/PutAllWithCallback.java     |   21 +-
 .../sockets/command/PutUserCredentials.java     |   21 +-
 .../cache/tier/sockets/command/Query.java       |   21 +-
 .../cache/tier/sockets/command/Query651.java    |   21 +-
 .../command/RegisterDataSerializers.java        |   21 +-
 .../sockets/command/RegisterInstantiators.java  |   21 +-
 .../tier/sockets/command/RegisterInterest.java  |   23 +-
 .../sockets/command/RegisterInterest61.java     |   23 +-
 .../sockets/command/RegisterInterestList.java   |   23 +-
 .../sockets/command/RegisterInterestList61.java |   23 +-
 .../sockets/command/RegisterInterestList66.java |   23 +-
 .../cache/tier/sockets/command/RemoveAll.java   |   21 +-
 .../tier/sockets/command/RemoveUserAuth.java    |   21 +-
 .../cache/tier/sockets/command/Request.java     |   23 +-
 .../tier/sockets/command/RequestEventValue.java |   20 +-
 .../tier/sockets/command/RollbackCommand.java   |   21 +-
 .../cache/tier/sockets/command/Size.java        |   21 +-
 .../tier/sockets/command/TXFailoverCommand.java |   21 +-
 .../command/TXSynchronizationCommand.java       |   21 +-
 .../sockets/command/UnregisterInterest.java     |   23 +-
 .../sockets/command/UnregisterInterestList.java |   23 +-
 .../command/UpdateClientNotification.java       |   21 +-
 .../doc-files/communication-architecture.fig    |    4 +-
 .../internal/cache/tier/sockets/package.html    |   16 +
 .../cache/tx/AbstractPeerTXRegionStub.java      |   21 +-
 .../internal/cache/tx/ClientTXRegionStub.java   |   21 +-
 .../internal/cache/tx/ClientTXStateStub.java    |   21 +-
 .../cache/tx/DistClientTXStateStub.java         |   19 +-
 .../internal/cache/tx/DistTxEntryEvent.java     |   86 +-
 .../internal/cache/tx/DistTxKeyInfo.java        |   18 +-
 .../cache/tx/DistributedTXRegionStub.java       |   21 +-
 .../cache/tx/PartitionedTXRegionStub.java       |   21 +-
 .../gemfire/internal/cache/tx/TXRegionStub.java |   21 +-
 .../cache/tx/TransactionalOperation.java        |   24 +-
 .../cache/versions/CompactVersionHolder.java    |   21 +-
 .../ConcurrentCacheModificationException.java   |   23 +-
 .../cache/versions/DiskRegionVersionVector.java |   21 +-
 .../internal/cache/versions/DiskVersionTag.java |   21 +-
 .../internal/cache/versions/RVVException.java   |   21 +-
 .../internal/cache/versions/RVVExceptionB.java  |   23 +-
 .../internal/cache/versions/RVVExceptionT.java  |   23 +-
 .../cache/versions/RegionVersionHolder.java     |   21 +-
 .../cache/versions/RegionVersionVector.java     |   29 +-
 .../cache/versions/VMRegionVersionVector.java   |   21 +-
 .../internal/cache/versions/VMVersionTag.java   |   21 +-
 .../internal/cache/versions/VersionHolder.java  |   21 +-
 .../internal/cache/versions/VersionSource.java  |   21 +-
 .../internal/cache/versions/VersionStamp.java   |   21 +-
 .../internal/cache/versions/VersionTag.java     |   21 +-
 .../internal/cache/vmotion/VMotionObserver.java |   20 +-
 .../cache/vmotion/VMotionObserverAdapter.java   |   20 +-
 .../cache/vmotion/VMotionObserverHolder.java    |   20 +-
 .../cache/wan/AbstractGatewaySender.java        |   50 +-
 .../AbstractGatewaySenderEventProcessor.java    |   22 +-
 .../AsyncEventQueueConfigurationException.java  |   21 +-
 .../internal/cache/wan/BatchException70.java    |   21 +-
 .../cache/wan/DistributedSystemListener.java    |   16 +
 .../cache/wan/GatewayEventFilterImpl.java       |   21 +-
 .../cache/wan/GatewayReceiverException.java     |   21 +-
 .../cache/wan/GatewayReceiverStats.java         |   21 +-
 .../cache/wan/GatewaySenderAdvisor.java         |   22 +-
 .../cache/wan/GatewaySenderAttributes.java      |   26 +-
 .../GatewaySenderConfigurationException.java    |   21 +-
 .../wan/GatewaySenderEventCallbackArgument.java |   20 +-
 .../GatewaySenderEventCallbackDispatcher.java   |   22 +-
 .../cache/wan/GatewaySenderEventDispatcher.java |   21 +-
 .../cache/wan/GatewaySenderEventImpl.java       |   30 +-
 .../cache/wan/GatewaySenderException.java       |   21 +-
 .../internal/cache/wan/GatewaySenderStats.java  |   21 +-
 .../cache/wan/InternalGatewaySenderFactory.java |   16 +
 .../cache/wan/TransportFilterServerSocket.java  |   21 +-
 .../cache/wan/TransportFilterSocket.java        |   21 +-
 .../cache/wan/TransportFilterSocketFactory.java |   21 +-
 .../internal/cache/wan/WANServiceProvider.java  |   16 +
 .../BucketRegionQueueUnavailableException.java  |   16 +
 ...rentParallelGatewaySenderEventProcessor.java |   23 +-
 .../ConcurrentParallelGatewaySenderQueue.java   |   21 +-
 .../ParallelGatewaySenderEventProcessor.java    |   23 +-
 .../parallel/ParallelGatewaySenderQueue.java    |  117 +-
 .../ParallelQueueBatchRemovalMessage.java       |   21 +-
 .../parallel/ParallelQueueRemovalMessage.java   |   21 +-
 .../cache/wan/parallel/RREventIDResolver.java   |   21 +-
 .../cache/wan/serial/BatchDestroyOperation.java |   21 +-
 ...urrentSerialGatewaySenderEventProcessor.java |   21 +-
 .../SerialGatewaySenderEventProcessor.java      |   24 +-
 .../wan/serial/SerialGatewaySenderQueue.java    |   23 +-
 .../serial/SerialSecondaryGatewayListener.java  |   21 +-
 .../internal/cache/wan/spi/WANFactory.java      |   16 +
 .../cache/xmlcache/AbstractXmlParser.java       |   21 +-
 .../cache/xmlcache/AsyncEventQueueCreation.java |   21 +-
 .../cache/xmlcache/BindingCreation.java         |   21 +-
 .../cache/xmlcache/BridgeServerCreation.java    |  249 -
 .../internal/cache/xmlcache/CacheCreation.java  |  170 +-
 .../cache/xmlcache/CacheServerCreation.java     |  257 +
 .../CacheTransactionManagerCreation.java        |   21 +-
 .../internal/cache/xmlcache/CacheXml.java       |   21 +-
 .../cache/xmlcache/CacheXmlGenerator.java       |   66 +-
 .../internal/cache/xmlcache/CacheXmlParser.java |  179 +-
 .../xmlcache/CacheXmlPropertyResolver.java      |   21 +-
 .../CacheXmlPropertyResolverHelper.java         |   21 +-
 .../cache/xmlcache/CacheXmlVersion.java         |   17 +-
 .../cache/xmlcache/ClientCacheCreation.java     |   26 +-
 .../cache/xmlcache/ClientHaQueueCreation.java   |   21 +-
 .../internal/cache/xmlcache/Declarable2.java    |   21 +-
 .../cache/xmlcache/DefaultEntityResolver2.java  |   23 +-
 .../xmlcache/DiskStoreAttributesCreation.java   |   21 +-
 .../cache/xmlcache/FunctionServiceCreation.java |   21 +-
 .../cache/xmlcache/GatewayReceiverCreation.java |   21 +-
 .../cache/xmlcache/GeodeEntityResolver.java     |   58 +
 .../cache/xmlcache/IndexCreationData.java       |   21 +-
 .../ParallelAsyncEventQueueCreation.java        |   21 +-
 .../xmlcache/ParallelGatewaySenderCreation.java |   21 +-
 .../cache/xmlcache/PivotalEntityResolver.java   |   23 +-
 .../cache/xmlcache/PropertyResolver.java        |   21 +-
 .../xmlcache/RegionAttributesCreation.java      |   30 +-
 .../internal/cache/xmlcache/RegionCreation.java |   21 +-
 .../cache/xmlcache/ResourceManagerCreation.java |   21 +-
 .../xmlcache/SerialAsyncEventQueueCreation.java |   21 +-
 .../xmlcache/SerialGatewaySenderCreation.java   |   21 +-
 .../cache/xmlcache/SerializerCreation.java      |   21 +-
 .../internal/cache/xmlcache/XmlGenerator.java   |   21 +-
 .../cache/xmlcache/XmlGeneratorUtils.java       |   21 +-
 .../internal/cache/xmlcache/XmlParser.java      |   21 +-
 .../internal/cache/xmlcache/package.html        |   16 +
 .../gemfire/internal/concurrent/AL.java         |   21 +-
 .../internal/concurrent/AtomicLong5.java        |   21 +-
 .../gemfire/internal/concurrent/Atomics.java    |   21 +-
 .../concurrent/CompactConcurrentHashSet2.java   |   16 +
 .../internal/concurrent/ConcurrentHashSet.java  |   21 +-
 .../gemfire/internal/concurrent/LI.java         |   21 +-
 .../internal/concurrent/MapCallback.java        |   21 +-
 .../internal/concurrent/MapCallbackAdapter.java |   21 +-
 .../gemfire/internal/concurrent/MapResult.java  |   21 +-
 .../internal/datasource/AbstractDataSource.java |   21 +-
 .../internal/datasource/AbstractPoolCache.java  |   25 +-
 .../ClientConnectionFactoryWrapper.java         |   21 +-
 .../internal/datasource/ConfigProperty.java     |   21 +-
 .../ConfiguredDataSourceProperties.java         |   21 +-
 .../ConnectionEventListenerAdaptor.java         |   21 +-
 .../datasource/ConnectionPoolCache.java         |   21 +-
 .../datasource/ConnectionPoolCacheImpl.java     |   21 +-
 .../internal/datasource/ConnectionProvider.java |   21 +-
 .../datasource/ConnectionProviderException.java |   21 +-
 .../datasource/DataSourceCreateException.java   |   21 +-
 .../internal/datasource/DataSourceFactory.java  |   21 +-
 .../datasource/DataSourceResources.java         |   21 +-
 .../FacetsJCAConnectionManagerImpl.java         |   21 +-
 .../datasource/GemFireBasicDataSource.java      |   23 +-
 .../datasource/GemFireConnPooledDataSource.java |   23 +-
 .../GemFireConnectionPoolManager.java           |   21 +-
 .../GemFireTransactionDataSource.java           |   21 +-
 .../datasource/JCAConnectionManagerImpl.java    |   21 +-
 .../datasource/ManagedPoolCacheImpl.java        |   21 +-
 .../internal/datasource/PoolException.java      |   21 +-
 .../internal/datasource/TranxPoolCacheImpl.java |   21 +-
 .../i18n/AbstractStringIdResourceBundle.java    |   26 +-
 .../gemfire/internal/i18n/LocalizedStrings.java | 4008 +++++++--------
 .../internal/i18n/ParentLocalizedStrings.java   | 4595 +++++++++---------
 .../gemfire/internal/i18n/StringId.java         |   58 +
 .../internal/io/CompositeOutputStream.java      |   21 +-
 .../internal/io/CompositePrintStream.java       |   21 +-
 .../gemfire/internal/io/TeeOutputStream.java    |   21 +-
 .../gemfire/internal/io/TeePrintStream.java     |   21 +-
 .../gemfire/internal/jndi/ContextImpl.java      |   21 +-
 .../jndi/InitialContextFactoryImpl.java         |   21 +-
 .../gemfire/internal/jndi/JNDIInvoker.java      |   21 +-
 .../gemfire/internal/jndi/NameParserImpl.java   |   21 +-
 .../gemfire/internal/jta/GlobalTransaction.java |   21 +-
 .../gemfire/internal/jta/TransactionImpl.java   |   21 +-
 .../internal/jta/TransactionManagerImpl.java    |   21 +-
 .../gemfire/internal/jta/TransactionUtils.java  |   21 +-
 .../internal/jta/UserTransactionImpl.java       |   21 +-
 .../gemstone/gemfire/internal/jta/XidImpl.java  |   21 +-
 .../gemfire/internal/lang/ClassUtils.java       |   20 +-
 .../gemstone/gemfire/internal/lang/Filter.java  |   18 +-
 .../gemfire/internal/lang/InOutParameter.java   |   20 +-
 .../gemfire/internal/lang/Initable.java         |   20 +-
 .../gemfire/internal/lang/Initializer.java      |   21 +-
 .../internal/lang/MutableIdentifiable.java      |   24 +-
 .../gemfire/internal/lang/ObjectUtils.java      |   20 +-
 .../gemfire/internal/lang/Orderable.java        |   20 +-
 .../gemstone/gemfire/internal/lang/Ordered.java |   20 +-
 .../gemfire/internal/lang/StringUtils.java      |  384 +-
 .../gemfire/internal/lang/SystemUtils.java      |   55 +-
 .../gemfire/internal/lang/ThreadUtils.java      |   20 +-
 .../gemfire/internal/logging/DateFormatter.java |   16 +
 .../internal/logging/DebugLogWriter.java        |   25 +-
 .../internal/logging/GemFireFormatter.java      |   21 +-
 .../internal/logging/GemFireHandler.java        |   21 +-
 .../gemfire/internal/logging/GemFireLevel.java  |   21 +-
 .../internal/logging/InternalLogWriter.java     |   21 +-
 .../internal/logging/LocalLogWriter.java        |   21 +-
 .../gemfire/internal/logging/LogConfig.java     |   16 +
 .../gemfire/internal/logging/LogFileParser.java |   29 +-
 .../gemfire/internal/logging/LogService.java    |  177 +-
 .../internal/logging/LogWriterFactory.java      |   16 +
 .../gemfire/internal/logging/LogWriterImpl.java |   23 +-
 .../internal/logging/LoggingThreadGroup.java    |   18 +-
 .../internal/logging/ManagerLogWriter.java      |   21 +-
 .../gemfire/internal/logging/MergeLogFiles.java |   21 +-
 .../gemfire/internal/logging/PureLogWriter.java |   23 +-
 .../logging/SecurityLocalLogWriter.java         |   23 +-
 .../internal/logging/SecurityLogConfig.java     |   16 +
 .../internal/logging/SecurityLogWriter.java     |   23 +-
 .../logging/SecurityManagerLogWriter.java       |   23 +-
 .../gemfire/internal/logging/SortLogFile.java   |   21 +-
 .../internal/logging/StandardErrorPrinter.java  |   16 +
 .../internal/logging/StandardOutputPrinter.java |   16 +
 .../internal/logging/log4j/AlertAppender.java   |   31 +-
 .../internal/logging/log4j/AppenderContext.java |   16 +
 .../internal/logging/log4j/ConfigLocator.java   |   16 +
 .../internal/logging/log4j/Configurator.java    |   94 +-
 .../internal/logging/log4j/FastLogger.java      |   66 +-
 .../internal/logging/log4j/GemFireLogger.java   |   16 +
 .../logging/log4j/LocalizedMessage.java         |   19 +-
 .../internal/logging/log4j/LogMarker.java       |   17 +-
 .../logging/log4j/LogWriterAppender.java        |   18 +-
 .../logging/log4j/LogWriterAppenders.java       |   16 +
 .../internal/logging/log4j/LogWriterLogger.java |   18 +-
 .../logging/log4j/ThreadIdPatternConverter.java |   16 +
 .../gemfire/internal/memcached/Command.java     |   21 +-
 .../internal/memcached/CommandProcessor.java    |   21 +-
 .../internal/memcached/ConnectionHandler.java   |   21 +-
 .../gemfire/internal/memcached/KeyWrapper.java  |   21 +-
 .../gemfire/internal/memcached/Reply.java       |   21 +-
 .../internal/memcached/RequestReader.java       |   21 +-
 .../internal/memcached/ResponseStatus.java      |   21 +-
 .../internal/memcached/ValueWrapper.java        |   21 +-
 .../memcached/commands/AbstractCommand.java     |   21 +-
 .../internal/memcached/commands/AddCommand.java |   21 +-
 .../memcached/commands/AddQCommand.java         |   16 +
 .../memcached/commands/AppendCommand.java       |   21 +-
 .../memcached/commands/AppendQCommand.java      |   16 +
 .../internal/memcached/commands/CASCommand.java |   21 +-
 .../memcached/commands/ClientError.java         |   21 +-
 .../memcached/commands/DecrementCommand.java    |   21 +-
 .../memcached/commands/DecrementQCommand.java   |   16 +
 .../memcached/commands/DeleteCommand.java       |   21 +-
 .../memcached/commands/DeleteQCommand.java      |   16 +
 .../memcached/commands/FlushAllCommand.java     |   21 +-
 .../memcached/commands/FlushAllQCommand.java    |   16 +
 .../internal/memcached/commands/GATCommand.java |   16 +
 .../memcached/commands/GATQCommand.java         |   16 +
 .../internal/memcached/commands/GetCommand.java |   28 +-
 .../memcached/commands/GetKCommand.java         |   16 +
 .../memcached/commands/GetKQCommand.java        |   16 +
 .../memcached/commands/GetQCommand.java         |   21 +-
 .../memcached/commands/IncrementCommand.java    |   21 +-
 .../memcached/commands/IncrementQCommand.java   |   16 +
 .../memcached/commands/NoOpCommand.java         |   21 +-
 .../memcached/commands/NotSupportedCommand.java |   21 +-
 .../memcached/commands/PrependCommand.java      |   21 +-
 .../memcached/commands/PrependQCommand.java     |   16 +
 .../memcached/commands/QuitCommand.java         |   21 +-
 .../memcached/commands/QuitQCommand.java        |   16 +
 .../memcached/commands/ReplaceCommand.java      |   21 +-
 .../memcached/commands/ReplaceQCommand.java     |   16 +
 .../internal/memcached/commands/SetCommand.java |   21 +-
 .../memcached/commands/SetQCommand.java         |   16 +
 .../memcached/commands/StatsCommand.java        |   21 +-
 .../memcached/commands/StorageCommand.java      |   21 +-
 .../memcached/commands/TouchCommand.java        |   21 +-
 .../memcached/commands/VerbosityCommand.java    |   21 +-
 .../memcached/commands/VersionCommand.java      |   21 +-
 .../modules/util/RegionConfiguration.java       |   21 +-
 .../gemfire/internal/net/SocketUtils.java       |   20 +-
 .../internal/offheap/AbstractStoredObject.java  |  107 +
 .../internal/offheap/ByteArrayMemoryChunk.java  |   16 +
 .../internal/offheap/ByteBufferMemoryChunk.java |   16 +
 .../gemfire/internal/offheap/Chunk.java         |  792 +++
 .../gemfire/internal/offheap/ChunkFactory.java  |   51 +
 .../gemfire/internal/offheap/ChunkType.java     |   30 +
 .../internal/offheap/ChunkWithHeapForm.java     |   40 +
 .../gemfire/internal/offheap/DataAsAddress.java |  126 +
 .../gemfire/internal/offheap/DataType.java      |   16 +
 .../gemfire/internal/offheap/Fragment.java      |  139 +
 .../internal/offheap/FreeListManager.java       |  821 ++++
 .../gemfire/internal/offheap/GemFireChunk.java  |   47 +
 .../internal/offheap/GemFireChunkFactory.java   |   52 +
 .../internal/offheap/GemFireChunkSlice.java     |   44 +
 .../internal/offheap/LifecycleListener.java     |   98 +
 .../internal/offheap/MemoryAllocator.java       |   19 +-
 .../gemfire/internal/offheap/MemoryBlock.java   |   17 +-
 .../internal/offheap/MemoryBlockNode.java       |  170 +
 .../gemfire/internal/offheap/MemoryChunk.java   |   16 +
 .../offheap/MemoryChunkWithRefCount.java        |   16 +
 .../internal/offheap/MemoryInspector.java       |   22 +-
 .../internal/offheap/MemoryUsageListener.java   |   16 +
 .../offheap/OffHeapCachedDeserializable.java    |   87 +-
 .../gemfire/internal/offheap/OffHeapHelper.java |   28 +-
 .../internal/offheap/OffHeapMemoryStats.java    |   16 +
 .../internal/offheap/OffHeapReference.java      |   56 -
 .../offheap/OffHeapRegionEntryHelper.java       |   75 +-
 .../internal/offheap/OffHeapStorage.java        |   49 +-
 .../offheap/OutOfOffHeapMemoryListener.java     |   16 +
 .../internal/offheap/RefCountChangeInfo.java    |  130 +
 .../internal/offheap/ReferenceCountHelper.java  |  254 +
 .../gemfire/internal/offheap/Releasable.java    |   16 +
 .../offheap/SimpleMemoryAllocatorImpl.java      | 3604 +-------------
 .../gemfire/internal/offheap/StoredObject.java  |   82 +-
 .../internal/offheap/SyncChunkStack.java        |  130 +
 .../internal/offheap/UnsafeMemoryChunk.java     |   26 +
 .../offheap/annotations/OffHeapIdentifier.java  |   16 +
 .../internal/offheap/annotations/Released.java  |   16 +
 .../internal/offheap/annotations/Retained.java  |   16 +
 .../offheap/annotations/Unretained.java         |   16 +
 .../com/gemstone/gemfire/internal/package.html  |   16 +
 .../internal/process/AttachProcessUtils.java    |   16 +
 .../process/BlockingProcessStreamReader.java    |   16 +
 ...usterConfigurationNotAvailableException.java |   16 +
 .../process/ConnectionFailedException.java      |   21 +-
 .../internal/process/ControlFileWatchdog.java   |   16 +
 .../process/ControlNotificationHandler.java     |   16 +
 .../internal/process/ControllableProcess.java   |   16 +
 .../process/FileAlreadyExistsException.java     |   21 +-
 .../process/FileControllerParameters.java       |   16 +
 .../internal/process/FileProcessController.java |   16 +
 .../process/LocalProcessController.java         |   21 +-
 .../internal/process/LocalProcessLauncher.java  |   21 +-
 .../process/MBeanControllerParameters.java      |   16 +
 .../process/MBeanInvocationFailedException.java |   21 +-
 .../process/MBeanProcessController.java         |   16 +
 .../internal/process/NativeProcessUtils.java    |   16 +
 .../process/NonBlockingProcessStreamReader.java |   16 +
 .../gemfire/internal/process/PidFile.java       |   16 +
 .../process/PidUnavailableException.java        |   21 +-
 .../internal/process/ProcessController.java     |   16 +
 .../process/ProcessControllerFactory.java       |   16 +
 .../process/ProcessControllerParameters.java    |   16 +
 .../process/ProcessLauncherContext.java         |   21 +-
 .../internal/process/ProcessStreamReader.java   |   21 +-
 .../ProcessTerminatedAbnormallyException.java   |   20 +-
 .../gemfire/internal/process/ProcessType.java   |   16 +
 .../gemfire/internal/process/ProcessUtils.java  |   21 +-
 .../gemfire/internal/process/StartupStatus.java |   18 +-
 .../internal/process/StartupStatusListener.java |   16 +
 .../UnableToControlProcessException.java        |   16 +
 .../AbstractSignalNotificationHandler.java      |   20 +-
 .../gemfire/internal/process/signal/Signal.java |   20 +-
 .../internal/process/signal/SignalEvent.java    |   20 +-
 .../internal/process/signal/SignalListener.java |   20 +-
 .../internal/process/signal/SignalType.java     |   20 +-
 .../internal/redis/ByteArrayWrapper.java        |   27 +-
 .../internal/redis/ByteToCommandDecoder.java    |   73 +-
 .../gemstone/gemfire/internal/redis/Coder.java  |  244 +-
 .../gemfire/internal/redis/Command.java         |   41 +-
 .../gemfire/internal/redis/DoubleWrapper.java   |   30 +-
 .../internal/redis/ExecutionHandlerContext.java |  218 +-
 .../gemfire/internal/redis/Executor.java        |   18 +-
 .../gemfire/internal/redis/Extendable.java      |   22 +-
 .../redis/RedisCommandParserException.java      |   16 +
 .../internal/redis/RedisCommandType.java        |   37 +
 .../gemfire/internal/redis/RedisConstants.java  |   22 +-
 .../gemfire/internal/redis/RedisDataType.java   |   27 +-
 .../redis/RedisDataTypeMismatchException.java   |   16 +
 .../gemfire/internal/redis/RegionCache.java     |  410 --
 .../internal/redis/RegionCreationException.java |   25 +-
 .../gemfire/internal/redis/RegionProvider.java  |  554 +++
 .../redis/executor/AbstractExecutor.java        |   28 +-
 .../redis/executor/AbstractScanExecutor.java    |   18 +-
 .../internal/redis/executor/AuthExecutor.java   |   54 +
 .../internal/redis/executor/DBSizeExecutor.java |   18 +-
 .../internal/redis/executor/DelExecutor.java    |   18 +-
 .../internal/redis/executor/EchoExecutor.java   |   18 +-
 .../internal/redis/executor/ExistsExecutor.java |   18 +-
 .../redis/executor/ExpirationExecutor.java      |   22 +-
 .../redis/executor/ExpireAtExecutor.java        |   20 +-
 .../internal/redis/executor/ExpireExecutor.java |   20 +-
 .../redis/executor/FlushAllExecutor.java        |   31 +-
 .../internal/redis/executor/KeysExecutor.java   |   23 +-
 .../internal/redis/executor/ListQuery.java      |   28 +-
 .../redis/executor/PExpireAtExecutor.java       |   16 +
 .../redis/executor/PExpireExecutor.java         |   16 +
 .../internal/redis/executor/PTTLExecutor.java   |   16 +
 .../redis/executor/PersistExecutor.java         |   18 +-
 .../internal/redis/executor/PingExecutor.java   |   18 +-
 .../internal/redis/executor/QuitExecutor.java   |   18 +-
 .../internal/redis/executor/ScanExecutor.java   |   20 +-
 .../redis/executor/ShutDownExecutor.java        |   16 +
 .../internal/redis/executor/SortedSetQuery.java |   52 +-
 .../internal/redis/executor/TTLExecutor.java    |   20 +-
 .../internal/redis/executor/TimeExecutor.java   |   16 +
 .../internal/redis/executor/TypeExecutor.java   |   18 +-
 .../internal/redis/executor/UnkownExecutor.java |   18 +-
 .../redis/executor/hash/HDelExecutor.java       |   18 +-
 .../redis/executor/hash/HExistsExecutor.java    |   16 +
 .../redis/executor/hash/HGetAllExecutor.java    |   23 +-
 .../redis/executor/hash/HGetExecutor.java       |   16 +
 .../redis/executor/hash/HIncrByExecutor.java    |   16 +
 .../executor/hash/HIncrByFloatExecutor.java     |   16 +
 .../redis/executor/hash/HKeysExecutor.java      |   23 +-
 .../redis/executor/hash/HLenExecutor.java       |   16 +
 .../redis/executor/hash/HMGetExecutor.java      |   16 +
 .../redis/executor/hash/HMSetExecutor.java      |   16 +
 .../redis/executor/hash/HScanExecutor.java      |   21 +-
 .../redis/executor/hash/HSetExecutor.java       |   16 +
 .../redis/executor/hash/HSetNXExecutor.java     |   16 +
 .../redis/executor/hash/HValsExecutor.java      |   25 +-
 .../redis/executor/hash/HashExecutor.java       |   22 +-
 .../internal/redis/executor/hll/Bits.java       |   16 +
 .../executor/hll/CardinalityMergeException.java |   18 +-
 .../redis/executor/hll/HllExecutor.java         |   18 +-
 .../redis/executor/hll/HyperLogLog.java         |   16 +
 .../redis/executor/hll/HyperLogLogPlus.java     |   18 +-
 .../internal/redis/executor/hll/IBuilder.java   |   18 +-
 .../redis/executor/hll/ICardinality.java        |   16 +
 .../internal/redis/executor/hll/MurmurHash.java |   18 +-
 .../redis/executor/hll/PFAddExecutor.java       |   18 +-
 .../redis/executor/hll/PFCountExecutor.java     |   18 +-
 .../redis/executor/hll/PFMergeExecutor.java     |   20 +-
 .../redis/executor/hll/RegisterSet.java         |   18 +-
 .../internal/redis/executor/hll/Varint.java     |   18 +-
 .../redis/executor/list/LIndexExecutor.java     |   20 +-
 .../redis/executor/list/LInsertExecutor.java    |   16 +
 .../redis/executor/list/LLenExecutor.java       |   18 +-
 .../redis/executor/list/LPopExecutor.java       |   16 +
 .../redis/executor/list/LPushExecutor.java      |   16 +
 .../redis/executor/list/LPushXExecutor.java     |   16 +
 .../redis/executor/list/LRangeExecutor.java     |   26 +-
 .../redis/executor/list/LRemExecutor.java       |   20 +-
 .../redis/executor/list/LSetExecutor.java       |   20 +-
 .../redis/executor/list/LTrimExecutor.java      |   42 +-
 .../redis/executor/list/ListExecutor.java       |   42 +-
 .../redis/executor/list/PopExecutor.java        |   39 +-
 .../redis/executor/list/PushExecutor.java       |   18 +-
 .../redis/executor/list/PushXExecutor.java      |   18 +-
 .../redis/executor/list/RPopExecutor.java       |   16 +
 .../redis/executor/list/RPushExecutor.java      |   16 +
 .../redis/executor/list/RPushXExecutor.java     |   16 +
 .../org/apache/hadoop/fs/GlobPattern.java       |  164 -
 .../redis/executor/set/SAddExecutor.java        |   18 +-
 .../redis/executor/set/SCardExecutor.java       |   18 +-
 .../redis/executor/set/SDiffExecutor.java       |   16 +
 .../redis/executor/set/SDiffStoreExecutor.java  |   16 +
 .../redis/executor/set/SInterExecutor.java      |   16 +
 .../redis/executor/set/SInterStoreExecutor.java |   16 +
 .../redis/executor/set/SIsMemberExecutor.java   |   18 +-
 .../redis/executor/set/SMembersExecutor.java    |   23 +-
 .../redis/executor/set/SMoveExecutor.java       |   18 +-
 .../redis/executor/set/SPopExecutor.java        |   20 +-
 .../redis/executor/set/SRandMemberExecutor.java |   20 +-
 .../redis/executor/set/SRemExecutor.java        |   18 +-
 .../redis/executor/set/SScanExecutor.java       |   20 +-
 .../redis/executor/set/SUnionExecutor.java      |   16 +
 .../redis/executor/set/SUnionStoreExecutor.java |   16 +
 .../redis/executor/set/SetExecutor.java         |   16 +
 .../redis/executor/set/SetOpExecutor.java       |   25 +-
 .../executor/sortedset/SortedSetExecutor.java   |   20 +-
 .../redis/executor/sortedset/ZAddExecutor.java  |   16 +
 .../redis/executor/sortedset/ZCardExecutor.java |   16 +
 .../executor/sortedset/ZCountExecutor.java      |   16 +
 .../executor/sortedset/ZIncrByExecutor.java     |   16 +
 .../executor/sortedset/ZLexCountExecutor.java   |   16 +
 .../executor/sortedset/ZRangeByLexExecutor.java |   18 +-
 .../sortedset/ZRangeByScoreExecutor.java        |   19 +-
 .../executor/sortedset/ZRangeExecutor.java      |   16 +
 .../redis/executor/sortedset/ZRankExecutor.java |   16 +
 .../redis/executor/sortedset/ZRemExecutor.java  |   18 +-
 .../sortedset/ZRemRangeByLexExecutor.java       |   19 +-
 .../sortedset/ZRemRangeByRankExecutor.java      |   20 +-
 .../sortedset/ZRemRangeByScoreExecutor.java     |   20 +-
 .../sortedset/ZRevRangeByScoreExecutor.java     |   16 +
 .../executor/sortedset/ZRevRangeExecutor.java   |   16 +
 .../executor/sortedset/ZRevRankExecutor.java    |   16 +
 .../redis/executor/sortedset/ZScanExecutor.java |   21 +-
 .../executor/sortedset/ZScoreExecutor.java      |   16 +
 .../redis/executor/string/AppendExecutor.java   |   18 +-
 .../redis/executor/string/BitCountExecutor.java |   18 +-
 .../redis/executor/string/BitOpExecutor.java    |   18 +-
 .../redis/executor/string/BitPosExecutor.java   |   18 +-
 .../redis/executor/string/DecrByExecutor.java   |   18 +-
 .../redis/executor/string/DecrExecutor.java     |   20 +-
 .../redis/executor/string/GetBitExecutor.java   |   18 +-
 .../redis/executor/string/GetExecutor.java      |   18 +-
 .../redis/executor/string/GetRangeExecutor.java |   18 +-
 .../redis/executor/string/GetSetExecutor.java   |   18 +-
 .../redis/executor/string/IncrByExecutor.java   |   18 +-
 .../executor/string/IncrByFloatExecutor.java    |   18 +-
 .../redis/executor/string/IncrExecutor.java     |   18 +-
 .../redis/executor/string/MGetExecutor.java     |   18 +-
 .../redis/executor/string/MSetExecutor.java     |   18 +-
 .../redis/executor/string/MSetNXExecutor.java   |   18 +-
 .../redis/executor/string/PSetEXExecutor.java   |   16 +
 .../redis/executor/string/SetBitExecutor.java   |   18 +-
 .../redis/executor/string/SetEXExecutor.java    |   20 +-
 .../redis/executor/string/SetExecutor.java      |   20 +-
 .../redis/executor/string/SetNXExecutor.java    |   18 +-
 .../redis/executor/string/SetRangeExecutor.java |   20 +-
 .../redis/executor/string/StringExecutor.java   |   22 +-
 .../redis/executor/string/StrlenExecutor.java   |   18 +-
 .../executor/transactions/DiscardExecutor.java  |   16 +
 .../executor/transactions/ExecExecutor.java     |   16 +
 .../executor/transactions/MultiExecutor.java    |   16 +
 .../transactions/TransactionExecutor.java       |   16 +
 .../executor/transactions/UnwatchExecutor.java  |   16 +
 .../executor/transactions/WatchExecutor.java    |   16 +
 .../redis/org/apache/hadoop/fs/GlobPattern.java |  164 +
 .../internal/security/AuthorizeRequest.java     |   21 +-
 .../internal/security/AuthorizeRequestPP.java   |   21 +-
 .../security/FilterPostAuthorization.java       |   21 +-
 .../security/FilterPreAuthorization.java        |   21 +-
 .../internal/security/ObjectWithAuthz.java      |   21 +-
 .../gemfire/internal/security/package.html      |   16 +
 .../internal/sequencelog/EntryLogger.java       |   23 +-
 .../gemfire/internal/sequencelog/GraphType.java |   21 +-
 .../internal/sequencelog/MembershipLogger.java  |   21 +-
 .../internal/sequencelog/MessageLogger.java     |   21 +-
 .../internal/sequencelog/RegionLogger.java      |   21 +-
 .../internal/sequencelog/SequenceLogger.java    |   21 +-
 .../sequencelog/SequenceLoggerImpl.java         |   21 +-
 .../internal/sequencelog/Transition.java        |   21 +-
 .../gemfire/internal/sequencelog/io/Filter.java |   21 +-
 .../sequencelog/io/GemfireLogConverter.java     |   21 +-
 .../internal/sequencelog/io/GraphReader.java    |   21 +-
 .../sequencelog/io/InputStreamReader.java       |   21 +-
 .../sequencelog/io/OutputStreamAppender.java    |   21 +-
 .../internal/sequencelog/model/Edge.java        |   23 +-
 .../internal/sequencelog/model/Graph.java       |   21 +-
 .../internal/sequencelog/model/GraphID.java     |   21 +-
 .../sequencelog/model/GraphReaderCallback.java  |   23 +-
 .../internal/sequencelog/model/GraphSet.java    |   21 +-
 .../internal/sequencelog/model/Vertex.java      |   21 +-
 .../visualization/text/TextDisplay.java         |   21 +-
 .../gemfire/internal/shared/NativeCalls.java    |   21 +-
 .../internal/shared/NativeCallsJNAImpl.java     |   21 +-
 .../internal/shared/NativeErrorException.java   |   21 +-
 .../gemfire/internal/shared/OSType.java         |   21 +-
 .../internal/shared/StringPrintWriter.java      |   21 +-
 .../internal/shared/TCPSocketOptions.java       |   21 +-
 .../internal/size/CachingSingleObjectSizer.java |   21 +-
 .../size/InstrumentationSingleObjectSizer.java  |   21 +-
 .../gemfire/internal/size/ObjectGraphSizer.java |   28 +-
 .../gemfire/internal/size/ObjectTraverser.java  |   28 +-
 .../internal/size/ReflectionObjectSizer.java    |   21 +-
 .../size/ReflectionSingleObjectSizer.java       |   25 +-
 .../internal/size/SingleObjectSizer.java        |   16 +
 .../internal/size/SizeClassOnceObjectSizer.java |   21 +-
 .../gemfire/internal/size/SizeOfUtil0.java      |   21 +-
 .../internal/size/WellKnownClassSizer.java      |   21 +-
 .../internal/statistics/CounterMonitor.java     |   21 +-
 .../internal/statistics/GaugeMonitor.java       |   21 +-
 .../statistics/IgnoreResourceException.java     |   21 +-
 .../MapBasedStatisticsNotification.java         |   21 +-
 .../internal/statistics/ResourceInstance.java   |   21 +-
 .../internal/statistics/ResourceType.java       |   21 +-
 .../internal/statistics/SampleCollector.java    |   25 +-
 .../internal/statistics/SampleHandler.java      |   21 +-
 .../internal/statistics/SimpleStatisticId.java  |   21 +-
 .../statistics/StatArchiveDescriptor.java       |   21 +-
 .../internal/statistics/StatArchiveHandler.java |   21 +-
 .../statistics/StatArchiveHandlerConfig.java    |   21 +-
 .../internal/statistics/StatMonitorHandler.java |   21 +-
 .../internal/statistics/StatisticId.java        |   21 +-
 .../statistics/StatisticNotFoundException.java  |   21 +-
 .../internal/statistics/StatisticsListener.java |   23 +-
 .../internal/statistics/StatisticsMonitor.java  |   21 +-
 .../statistics/StatisticsNotification.java      |   21 +-
 .../internal/statistics/StatisticsSampler.java  |   21 +-
 .../internal/statistics/ValueMonitor.java       |   21 +-
 .../gemfire/internal/statistics/package.html    |   16 +
 .../stats50/Atomic50StatisticsImpl.java         |   21 +-
 .../gemfire/internal/stats50/VMStats50.java     |   21 +-
 .../gemfire/internal/tcp/BaseMsgStreamer.java   |   21 +-
 .../gemstone/gemfire/internal/tcp/Buffers.java  |   21 +-
 .../internal/tcp/ByteBufferInputStream.java     |   23 +-
 .../gemfire/internal/tcp/ConnectExceptions.java |   21 +-
 .../gemfire/internal/tcp/Connection.java        |  324 +-
 .../internal/tcp/ConnectionException.java       |   21 +-
 .../gemfire/internal/tcp/ConnectionTable.java   |  214 +-
 .../gemfire/internal/tcp/DirectReplySender.java |   21 +-
 .../tcp/ImmutableByteBufferInputStream.java     |   23 +-
 .../internal/tcp/MemberShunnedException.java    |   28 +-
 .../gemfire/internal/tcp/MsgDestreamer.java     |   21 +-
 .../gemfire/internal/tcp/MsgIdGenerator.java    |   21 +-
 .../gemfire/internal/tcp/MsgOutputStream.java   |   25 +-
 .../gemfire/internal/tcp/MsgReader.java         |   21 +-
 .../gemfire/internal/tcp/MsgStreamer.java       |   21 +-
 .../gemfire/internal/tcp/MsgStreamerList.java   |   21 +-
 .../gemfire/internal/tcp/NIOMsgReader.java      |   21 +-
 .../gemfire/internal/tcp/OioMsgReader.java      |   21 +-
 .../internal/tcp/ReenteredConnectException.java |   21 +-
 .../gemfire/internal/tcp/ServerDelegate.java    |   26 +-
 .../com/gemstone/gemfire/internal/tcp/Stub.java |  155 -
 .../gemfire/internal/tcp/TCPConduit.java        |  303 +-
 .../tcp/VersionedByteBufferInputStream.java     |   21 +-
 .../internal/tcp/VersionedMsgStreamer.java      |   21 +-
 .../gemstone/gemfire/internal/tcp/package.html  |   16 +
 .../gfsh/aggregator/AggregateFunction.java      |   67 -
 .../tools/gfsh/aggregator/AggregateResults.java |  126 -
 .../aggregator/AggregatorPartitionFunction.java |   54 -
 .../tools/gfsh/app/CommandExecutable.java       |    8 -
 .../gemfire/internal/tools/gfsh/app/Gfsh.java   | 2027 --------
 .../internal/tools/gfsh/app/GfshVersion.java    |  455 --
 .../internal/tools/gfsh/app/Nextable.java       |    8 -
 .../tools/gfsh/app/ServerExecutable.java        |   11 -
 .../app/aggregator/AggregateFunctionTask.java   |   62 -
 .../tools/gfsh/app/aggregator/Aggregator.java   |  549 ---
 .../app/aggregator/AggregatorException.java     |   50 -
 .../gfsh/app/aggregator/AggregatorPeer.java     |  254 -
 .../functions/util/LocalRegionInfoFunction.java |  235 -
 .../functions/util/RegionCreateFunction.java    |   81 -
 .../functions/util/RegionDestroyFunction.java   |   86 -
 .../tools/gfsh/app/cache/CacheBase.java         |  178 -
 .../gfsh/app/cache/InstantiatorClassLoader.java |  101 -
 .../tools/gfsh/app/cache/data/GenericMap.java   |  196 -
 .../gfsh/app/cache/data/GenericMessage.java     |  807 ---
 .../app/cache/data/InvalidTypeException.java    |   28 -
 .../tools/gfsh/app/cache/data/ListMap.java      |  197 -
 .../gfsh/app/cache/data/ListMapMessage.java     |  615 ---
 .../tools/gfsh/app/cache/data/ListMessage.java  |  594 ---
 .../tools/gfsh/app/cache/data/Listable.java     |   45 -
 .../tools/gfsh/app/cache/data/MapMessage.java   |  629 ---
 .../tools/gfsh/app/cache/data/Mappable.java     |   50 -
 .../tools/gfsh/app/cache/index/EntryMap.java    |   80 -
 .../tools/gfsh/app/cache/index/IndexInfo.java   |   52 -
 .../tools/gfsh/app/cache/index/Indexer.java     |   13 -
 .../gfsh/app/cache/index/IndexerManager.java    |   32 -
 .../gfsh/app/cache/index/LookupService.java     |  352 --
 .../app/cache/index/LookupServiceException.java |   64 -
 .../gfsh/app/cache/index/task/ForceGCTask.java  |   32 -
 .../app/cache/index/task/IndexInfoTask.java     |  131 -
 .../app/cache/index/task/QuerySizeTask.java     |  134 -
 .../gfsh/app/cache/index/task/QueryTask.java    |  323 --
 .../tools/gfsh/app/command/CommandClient.java   |  417 --
 .../gfsh/app/command/CommandException.java      |   31 -
 .../app/command/CommandResultsListener.java     |   15 -
 .../tools/gfsh/app/command/task/EchoTask.java   |   63 -
 .../task/PartitionedRegionAttributeTask.java    |  205 -
 .../gfsh/app/command/task/QueryResults.java     |  123 -
 .../tools/gfsh/app/command/task/QueryTask.java  |  522 --
 .../task/RefreshAggregatorRegionTask.java       |   53 -
 .../gfsh/app/command/task/RegionClearTask.java  |   85 -
 .../gfsh/app/command/task/RegionCreateTask.java |  162 -
 .../app/command/task/RegionDestroyTask.java     |   97 -
 .../gfsh/app/command/task/RegionPathTask.java   |  156 -
 .../gfsh/app/command/task/RegionSizeTask.java   |   70 -
 .../gfsh/app/command/task/data/MemberInfo.java  |   79 -
 .../task/data/PartitionAttributeInfo.java       |  124 -
 .../command/task/data/RegionAttributeInfo.java  |  286 --
 .../internal/tools/gfsh/app/commands/bcp.java   |  527 --
 .../internal/tools/gfsh/app/commands/cd.java    |   79 -
 .../tools/gfsh/app/commands/classloader.java    |  323 --
 .../internal/tools/gfsh/app/commands/clear.java |  197 -
 .../tools/gfsh/app/commands/connect.java        |  124 -
 .../internal/tools/gfsh/app/commands/db.java    |  312 --
 .../internal/tools/gfsh/app/commands/debug.java |   48 -
 .../tools/gfsh/app/commands/deploy.java         |  271 --
 .../internal/tools/gfsh/app/commands/echo.java  |   59 -
 .../internal/tools/gfsh/app/commands/fetch.java |   48 -
 .../internal/tools/gfsh/app/commands/gc.java    |   93 -
 .../internal/tools/gfsh/app/commands/get.java   |  143 -
 .../internal/tools/gfsh/app/commands/help.java  |   40 -
 .../internal/tools/gfsh/app/commands/index.java |  395 --
 .../internal/tools/gfsh/app/commands/key.java   |   67 -
 .../internal/tools/gfsh/app/commands/local.java |  101 -
 .../internal/tools/gfsh/app/commands/ls.java    |  584 ---
 .../internal/tools/gfsh/app/commands/mkdir.java |  253 -
 .../internal/tools/gfsh/app/commands/next.java  |   63 -
 .../tools/gfsh/app/commands/optional/look.java  |  166 -
 .../tools/gfsh/app/commands/optional/perf.java  |  150 -
 .../internal/tools/gfsh/app/commands/pr.java    |  209 -
 .../tools/gfsh/app/commands/property.java       |   85 -
 .../internal/tools/gfsh/app/commands/put.java   |  490 --
 .../internal/tools/gfsh/app/commands/pwd.java   |   37 -
 .../tools/gfsh/app/commands/rebalance.java      |  186 -
 .../tools/gfsh/app/commands/refresh.java        |   67 -
 .../internal/tools/gfsh/app/commands/rm.java    |  175 -
 .../internal/tools/gfsh/app/commands/rmdir.java |  249 -
 .../tools/gfsh/app/commands/select.java         |  139 -
 .../internal/tools/gfsh/app/commands/show.java  |  240 -
 .../internal/tools/gfsh/app/commands/size.java  |  281 --
 .../internal/tools/gfsh/app/commands/value.java |   48 -
 .../internal/tools/gfsh/app/commands/which.java |  189 -
 .../internal/tools/gfsh/app/commands/zone.java  |   49 -
 .../tools/gfsh/app/function/GfshData.java       |   86 -
 .../tools/gfsh/app/function/GfshFunction.java   |  149 -
 .../tools/gfsh/app/function/command/clear.java  |   95 -
 .../tools/gfsh/app/function/command/deploy.java |  111 -
 .../tools/gfsh/app/function/command/gc.java     |   49 -
 .../tools/gfsh/app/function/command/index.java  |  199 -
 .../tools/gfsh/app/function/command/ls.java     |   80 -
 .../tools/gfsh/app/function/command/pr.java     |  118 -
 .../gfsh/app/function/command/rebalance.java    |  129 -
 .../tools/gfsh/app/function/command/rm.java     |   43 -
 .../tools/gfsh/app/function/command/which.java  |  159 -
 .../tools/gfsh/app/misc/util/ClassFinder.java   |  209 -
 .../gfsh/app/misc/util/DataSerializerEx.java    |   98 -
 .../app/misc/util/QueueDispatcherListener.java  |   11 -
 .../app/misc/util/QueueDispatcherThread.java    |   87 -
 .../gfsh/app/misc/util/ReflectionUtil.java      |  303 --
 .../tools/gfsh/app/misc/util/StringUtil.java    |  142 -
 .../app/misc/util/SystemClassPathManager.java   |  171 -
 .../gfsh/app/pogo/InvalidKeyException.java      |   34 -
 .../internal/tools/gfsh/app/pogo/KeyType.java   |   96 -
 .../tools/gfsh/app/pogo/KeyTypeManager.java     |  199 -
 .../internal/tools/gfsh/app/pogo/MapLite.java   | 1136 -----
 .../tools/gfsh/app/pogo/MapLiteSerializer.java  |  338 --
 .../internal/tools/gfsh/app/util/DBUtil.java    | 1094 -----
 .../tools/gfsh/app/util/DBUtilException.java    |   48 -
 .../tools/gfsh/app/util/GfshResultsBag.java     |  600 ---
 .../tools/gfsh/app/util/ObjectUtil.java         |  164 -
 .../tools/gfsh/app/util/OutputUtil.java         |  255 -
 .../internal/tools/gfsh/app/util/PrintUtil.java | 1683 -------
 .../tools/gfsh/app/util/SimplePrintUtil.java    | 1366 ------
 .../tools/gfsh/command/AbstractCommandTask.java |  136 -
 .../tools/gfsh/command/CommandResults.java      |  119 -
 .../tools/gfsh/command/CommandTask.java         |   23 -
 .../internal/tools/gfsh/util/RegionUtil.java    |  829 ----
 .../internal/util/AbortableTaskService.java     |   21 +-
 .../gemfire/internal/util/ArrayUtils.java       |   21 +-
 .../gemfire/internal/util/BlobHelper.java       |   23 +-
 .../gemfire/internal/util/Breadcrumbs.java      |   23 +-
 .../gemstone/gemfire/internal/util/Bytes.java   |   21 +-
 .../gemfire/internal/util/Callable.java         |   21 +-
 .../gemfire/internal/util/CollectionUtils.java  |   21 +-
 .../gemfire/internal/util/DebuggerSupport.java  |   25 +-
 .../gemfire/internal/util/DelayedAction.java    |   21 +-
 .../com/gemstone/gemfire/internal/util/Hex.java |   21 +-
 .../gemstone/gemfire/internal/util/IOUtils.java |   21 +-
 .../internal/util/JavaCommandBuilder.java       |   21 +-
 .../gemfire/internal/util/LogFileUtils.java     |   21 +-
 .../internal/util/ObjectIntProcedure.java       |   16 +
 .../gemfire/internal/util/ObjectProcedure.java  |   14 -
 .../gemfire/internal/util/PasswordUtil.java     |   20 +-
 .../gemfire/internal/util/PluckStacks.java      |   95 +-
 .../gemfire/internal/util/PrimeFinder.java      |  159 -
 .../internal/util/SingletonCallable.java        |   16 +
 .../gemfire/internal/util/SingletonValue.java   |   24 +-
 .../internal/util/StackTraceCollector.java      |   21 +-
 .../gemfire/internal/util/StopWatch.java        |   21 +-
 .../internal/util/SunAPINotFoundException.java  |   20 +-
 .../gemfire/internal/util/TransformUtils.java   |   21 +-
 .../gemfire/internal/util/Transformer.java      |   21 +-
 .../gemfire/internal/util/Versionable.java      |   21 +-
 .../internal/util/VersionedArrayList.java       |   20 +-
 .../util/concurrent/CopyOnWriteHashMap.java     |   67 +-
 .../util/concurrent/CopyOnWriteWeakHashMap.java |   33 +-
 .../CustomEntryConcurrentHashMap.java           |   29 +-
 .../internal/util/concurrent/FutureResult.java  |   21 +-
 .../util/concurrent/ReentrantSemaphore.java     |   21 +-
 .../util/concurrent/SemaphoreReadWriteLock.java |   21 +-
 .../util/concurrent/StoppableCondition.java     |   21 +-
 .../concurrent/StoppableCountDownLatch.java     |   21 +-
 .../concurrent/StoppableCountDownOrUpLatch.java |   21 +-
 .../concurrent/StoppableNonReentrantLock.java   |   21 +-
 .../util/concurrent/StoppableReadWriteLock.java |   16 +
 .../util/concurrent/StoppableReentrantLock.java |   21 +-
 .../StoppableReentrantReadWriteLock.java        |   21 +-
 .../lang/AttachAPINotFoundException.java        |   20 +-
 .../com/gemstone/gemfire/lang/Identifiable.java |   20 +-
 .../management/AlreadyRunningException.java     |   21 +-
 .../management/AsyncEventQueueMXBean.java       |   20 +-
 .../gemfire/management/CacheServerMXBean.java   |   20 +-
 .../gemfire/management/ClientHealthStatus.java  |   20 +-
 .../gemfire/management/ClientQueueDetail.java   |   21 +-
 .../DependenciesNotFoundException.java          |   20 +-
 .../gemfire/management/DiskBackupResult.java    |   20 +-
 .../gemfire/management/DiskBackupStatus.java    |   20 +-
 .../gemfire/management/DiskMetrics.java         |   20 +-
 .../gemfire/management/DiskStoreMXBean.java     |   20 +-
 .../DistributedLockServiceMXBean.java           |   20 +-
 .../management/DistributedRegionMXBean.java     |   20 +-
 .../management/DistributedSystemMXBean.java     |   20 +-
 .../management/EvictionAttributesData.java      |   20 +-
 .../FixedPartitionAttributesData.java           |   21 +-
 .../management/GatewayReceiverMXBean.java       |   20 +-
 .../gemfire/management/GatewaySenderMXBean.java |   20 +-
 .../gemfire/management/GemFireProperties.java   |   20 +-
 .../gemfire/management/JMXNotificationType.java |   28 +-
 .../management/JMXNotificationUserData.java     |   28 +-
 .../gemstone/gemfire/management/JVMMetrics.java |   20 +-
 .../gemfire/management/LocatorMXBean.java       |   20 +-
 .../gemfire/management/LockServiceMXBean.java   |   20 +-
 .../gemfire/management/ManagementException.java |   20 +-
 .../gemfire/management/ManagementService.java   |   21 +-
 .../gemfire/management/ManagerMXBean.java       |   20 +-
 .../gemfire/management/MemberMXBean.java        |   20 +-
 .../management/MembershipAttributesData.java    |   20 +-
 .../gemfire/management/NetworkMetrics.java      |   20 +-
 .../gemstone/gemfire/management/OSMetrics.java  |   20 +-
 .../management/PartitionAttributesData.java     |   20 +-
 .../management/PersistentMemberDetails.java     |   20 +-
 .../management/RegionAttributesData.java        |   20 +-
 .../gemfire/management/RegionMXBean.java        |   20 +-
 .../gemfire/management/ServerLoadData.java      |   20 +-
 .../gemfire/management/cli/CliMetaData.java     |   20 +-
 .../cli/CommandProcessingException.java         |   20 +-
 .../gemfire/management/cli/CommandService.java  |   20 +-
 .../management/cli/CommandServiceException.java |   20 +-
 .../management/cli/CommandStatement.java        |   20 +-
 .../gemfire/management/cli/ConverterHint.java   |   21 +-
 .../gemstone/gemfire/management/cli/Result.java |   21 +-
 .../gemfire/management/cli/package.html         |   16 +
 .../gemfire/management/internal/AgentUtil.java  |   56 +-
 .../management/internal/AlertDetails.java       |   21 +-
 .../management/internal/ArrayConverter.java     |   20 +-
 .../internal/BaseManagementService.java         |   21 +-
 .../internal/CollectionConverter.java           |   22 +-
 .../management/internal/CompositeConverter.java |   20 +-
 .../management/internal/EnumConverter.java      |   20 +-
 .../management/internal/FederatingManager.java  |   26 +-
 .../internal/FederationComponent.java           |   20 +-
 .../management/internal/FilterChain.java        |   20 +-
 .../management/internal/FilterParam.java        |   22 +-
 .../management/internal/IdentityConverter.java  |   20 +-
 .../management/internal/JettyHelper.java        |   21 +-
 .../management/internal/JmxManagerAdvisee.java  |   21 +-
 .../management/internal/JmxManagerAdvisor.java  |   23 +-
 .../management/internal/JmxManagerLocator.java  |   21 +-
 .../internal/JmxManagerLocatorRequest.java      |   31 +-
 .../internal/JmxManagerLocatorResponse.java     |   20 +-
 .../management/internal/LocalFilterChain.java   |   20 +-
 .../management/internal/LocalManager.java       |   20 +-
 .../management/internal/MBeanJMXAdapter.java    |   22 +-
 .../management/internal/MBeanProxyFactory.java  |   22 +-
 .../internal/MBeanProxyInfoRepository.java      |   22 +-
 .../internal/MBeanProxyInvocationHandler.java   |   23 +-
 .../internal/MXBeanProxyInvocationHandler.java  |   20 +-
 .../management/internal/ManagementAgent.java    |   21 +-
 .../internal/ManagementCacheListener.java       |   22 +-
 .../internal/ManagementConstants.java           |   20 +-
 .../management/internal/ManagementFunction.java |   20 +-
 .../internal/ManagementMembershipListener.java  |   24 +-
 .../internal/ManagementResourceRepo.java        |   20 +-
 .../management/internal/ManagementStrings.java  |  133 +-
 .../gemfire/management/internal/Manager.java    |   21 +-
 .../internal/ManagerStartupMessage.java         |   21 +-
 .../management/internal/MemberMessenger.java    |   20 +-
 .../internal/MonitoringRegionCacheListener.java |   20 +-
 .../internal/NotificationBroadCasterProxy.java  |   20 +-
 .../internal/NotificationCacheListener.java     |   20 +-
 .../management/internal/NotificationHub.java    |   22 +-
 .../internal/NotificationHubClient.java         |   20 +-
 .../management/internal/NotificationKey.java    |   20 +-
 .../gemfire/management/internal/OpenMethod.java |   22 +-
 .../management/internal/OpenTypeConverter.java  |   20 +-
 .../management/internal/OpenTypeUtil.java       |   20 +-
 .../gemfire/management/internal/ProxyInfo.java  |   20 +-
 .../management/internal/ProxyInterface.java     |   20 +-
 .../management/internal/ProxyListener.java      |   20 +-
 .../management/internal/RemoteFilterChain.java  |   20 +-
 .../gemfire/management/internal/RestAgent.java  |   21 +-
 .../gemfire/management/internal/SSLUtil.java    |   16 +
 .../management/internal/StringBasedFilter.java  |   20 +-
 .../internal/SystemManagementService.java       |   28 +-
 .../management/internal/TableConverter.java     |   22 +-
 .../internal/beans/AggregateHandler.java        |   20 +-
 .../internal/beans/AsyncEventQueueMBean.java    |   20 +-
 .../beans/AsyncEventQueueMBeanBridge.java       |   20 +-
 .../internal/beans/BeanUtilFuncs.java           |   20 +-
 .../internal/beans/CacheServerBridge.java       |   40 +-
 .../internal/beans/CacheServerMBean.java        |   20 +-
 .../internal/beans/DiskRegionBridge.java        |   22 +-
 .../internal/beans/DiskStoreMBean.java          |   20 +-
 .../internal/beans/DiskStoreMBeanBridge.java    |   20 +-
 .../beans/DistributedLockServiceBridge.java     |   20 +-
 .../beans/DistributedLockServiceMBean.java      |   20 +-
 .../internal/beans/DistributedRegionBridge.java |   20 +-
 .../internal/beans/DistributedRegionMBean.java  |   20 +-
 .../internal/beans/DistributedSystemBridge.java |   20 +-
 .../internal/beans/DistributedSystemMBean.java  |   20 +-
 .../internal/beans/GatewayReceiverMBean.java    |   20 +-
 .../beans/GatewayReceiverMBeanBridge.java       |   24 +-
 .../internal/beans/GatewaySenderMBean.java      |   20 +-
 .../beans/GatewaySenderMBeanBridge.java         |   20 +-
 .../internal/beans/HDFSRegionBridge.java        |   20 +-
 .../management/internal/beans/LocatorMBean.java |   20 +-
 .../internal/beans/LocatorMBeanBridge.java      |   20 +-
 .../internal/beans/LockServiceMBean.java        |   20 +-
 .../internal/beans/LockServiceMBeanBridge.java  |   22 +-
 .../internal/beans/MBeanAggregator.java         |   22 +-
 .../internal/beans/ManagementAdapter.java       |   64 +-
 .../internal/beans/ManagementListener.java      |   20 +-
 .../management/internal/beans/ManagerMBean.java |   20 +-
 .../internal/beans/ManagerMBeanBridge.java      |   20 +-
 .../management/internal/beans/MemberMBean.java  |   20 +-
 .../internal/beans/MemberMBeanBridge.java       |   36 +-
 .../internal/beans/MetricsCalculator.java       |   20 +-
 .../internal/beans/PartitionedRegionBridge.java |   20 +-
 .../internal/beans/QueryDataFunction.java       |   20 +-
 .../management/internal/beans/RegionMBean.java  |   20 +-
 .../internal/beans/RegionMBeanBridge.java       |   20 +-
 .../beans/RegionMBeanCompositeDataFactory.java  |   20 +-
 .../internal/beans/SequenceNumber.java          |   20 +-
 .../management/internal/beans/ServerBridge.java |   27 +-
 .../stats/AggregateRegionStatsMonitor.java      |   20 +-
 .../internal/beans/stats/GCStatsMonitor.java    |   20 +-
 .../GatewayReceiverClusterStatsMonitor.java     |   20 +-
 .../stats/GatewaySenderClusterStatsMonitor.java |   20 +-
 .../stats/IntegerStatsDeltaAggregator.java      |   23 +-
 .../beans/stats/LongStatsDeltaAggregator.java   |   23 +-
 .../internal/beans/stats/MBeanStatsMonitor.java |   20 +-
 .../beans/stats/MemberClusterStatsMonitor.java  |   20 +-
 .../beans/stats/MemberLevelDiskMonitor.java     |   20 +-
 .../beans/stats/RegionClusterStatsMonitor.java  |   20 +-
 .../beans/stats/ServerClusterStatsMonitor.java  |   20 +-
 .../internal/beans/stats/StatType.java          |   20 +-
 .../internal/beans/stats/StatsAggregator.java   |   20 +-
 .../beans/stats/StatsAverageLatency.java        |   20 +-
 .../internal/beans/stats/StatsKey.java          |   20 +-
 .../internal/beans/stats/StatsLatency.java      |   20 +-
 .../internal/beans/stats/StatsRate.java         |   20 +-
 .../internal/beans/stats/VMStatsMonitor.java    |   20 +-
 .../cli/AbstractCliAroundInterceptor.java       |   20 +-
 .../internal/cli/CliAroundInterceptor.java      |   20 +-
 .../management/internal/cli/CliUtil.java        |   20 +-
 .../management/internal/cli/CommandManager.java |   20 +-
 .../management/internal/cli/CommandRequest.java |   21 +-
 .../internal/cli/CommandResponse.java           |   22 +-
 .../internal/cli/CommandResponseBuilder.java    |   20 +-
 .../internal/cli/CommandResponseWriter.java     |   20 +-
 .../internal/cli/GfshParseResult.java           |   20 +-
 .../management/internal/cli/GfshParser.java     |   37 +-
 .../management/internal/cli/Launcher.java       |   20 +-
 .../management/internal/cli/LogWrapper.java     |   20 +-
 .../internal/cli/MultipleValueAdapter.java      |   21 +-
 .../internal/cli/MultipleValueConverter.java    |   21 +-
 .../internal/cli/annotation/CliArgument.java    |   20 +-
 .../cli/commands/AbstractCommandsSupport.java   |   20 +-
 .../internal/cli/commands/ClientCommands.java   |   64 +-
 .../internal/cli/commands/ConfigCommands.java   |   20 +-
 .../CreateAlterDestroyRegionCommands.java       |   41 +-
 .../internal/cli/commands/DataCommands.java     |   20 +-
 .../internal/cli/commands/DeployCommands.java   |   20 +-
 .../cli/commands/DiskStoreCommands.java         |   46 +-
 .../cli/commands/DurableClientCommands.java     |   20 +-
 ...ExportImportSharedConfigurationCommands.java |   21 +-
 .../internal/cli/commands/FunctionCommands.java |   20 +-
 .../internal/cli/commands/GfshHelpCommands.java |   20 +-
 .../cli/commands/HDFSStoreCommands.java         |  718 ---
 .../internal/cli/commands/IndexCommands.java    |   20 +-
 .../cli/commands/LauncherLifecycleCommands.java |   25 +-
 .../internal/cli/commands/MemberCommands.java   |   20 +-
 .../cli/commands/MiscellaneousCommands.java     |   58 +-
 .../internal/cli/commands/PDXCommands.java      |   16 +
 .../internal/cli/commands/QueueCommands.java    |   24 +-
 .../internal/cli/commands/RegionCommands.java   |   20 +-
 .../internal/cli/commands/ShellCommands.java    |   27 +-
 .../internal/cli/commands/StatusCommands.java   |   21 +-
 .../internal/cli/commands/WanCommands.java      |   21 +-
 .../cli/commands/dto/RegionAttributesInfo.java  |   21 +-
 .../cli/commands/dto/RegionDetails.java         |   21 +-
 .../cli/commands/dto/RegionMemberDetails.java   |   21 +-
 .../cli/converters/BooleanConverter.java        |   22 +-
 .../ClusterMemberIdNameConverter.java           |   20 +-
 .../converters/ConnectionEndpointConverter.java |   21 +-
 .../internal/cli/converters/DirConverter.java   |   22 +-
 .../cli/converters/DirPathConverter.java        |   20 +-
 .../cli/converters/DiskStoreNameConverter.java  |   20 +-
 .../internal/cli/converters/EnumConverter.java  |   20 +-
 .../cli/converters/FilePathConverter.java       |   20 +-
 .../cli/converters/FilePathStringConverter.java |   20 +-
 .../converters/GatewayReceiverIdsConverter.java |   21 +-
 .../converters/GatewaySenderIdConverter.java    |   20 +-
 .../cli/converters/HdfsStoreNameConverter.java  |   88 -
 .../internal/cli/converters/HelpConverter.java  |   20 +-
 .../cli/converters/HintTopicConverter.java      |   20 +-
 .../cli/converters/IndexTypeConverter.java      |   21 +-
 .../LocatorDiscoveryConfigConverter.java        |   20 +-
 .../cli/converters/LocatorIdNameConverter.java  |   20 +-
 .../cli/converters/LogLevelConverter.java       |   20 +-
 .../cli/converters/MemberGroupConverter.java    |   20 +-
 .../cli/converters/MemberIdNameConverter.java   |   20 +-
 .../cli/converters/RegionPathConverter.java     |   20 +-
 .../cli/converters/StringArrayConverter.java    |   20 +-
 .../cli/converters/StringListConverter.java     |   20 +-
 .../cli/domain/AsyncEventQueueDetails.java      |   21 +-
 .../internal/cli/domain/CacheServerInfo.java    |   21 +-
 .../cli/domain/ConnectToLocatorResult.java      |   20 +-
 .../internal/cli/domain/DataCommandRequest.java |   21 +-
 .../internal/cli/domain/DataCommandResult.java  |   21 +-
 .../internal/cli/domain/DiskStoreDetails.java   |   18 +-
 .../cli/domain/DurableCqNamesResult.java        |   21 +-
 .../cli/domain/EvictionAttributesInfo.java      |   21 +-
 .../domain/FixedPartitionAttributesInfo.java    |   21 +-
 .../internal/cli/domain/IndexDetails.java       |   20 +-
 .../internal/cli/domain/IndexInfo.java          |   21 +-
 .../cli/domain/MemberConfigurationInfo.java     |   21 +-
 .../internal/cli/domain/MemberInformation.java  |   20 +-
 .../internal/cli/domain/MemberResult.java       |   21 +-
 .../cli/domain/PartitionAttributesInfo.java     |   21 +-
 .../cli/domain/RegionAttributesInfo.java        |   21 +-
 .../internal/cli/domain/RegionDescription.java  |   21 +-
 .../cli/domain/RegionDescriptionPerMember.java  |   21 +-
 .../internal/cli/domain/RegionInformation.java  |   20 +-
 .../cli/domain/StackTracesPerMember.java        |   21 +-
 .../cli/domain/SubscriptionQueueSizeResult.java |   21 +-
 .../cli/exceptions/CliCommandException.java     |   20 +-
 .../exceptions/CliCommandInvalidException.java  |   20 +-
 .../CliCommandMultiModeOptionException.java     |   16 +
 .../CliCommandNotAvailableException.java        |   20 +-
 .../exceptions/CliCommandOptionException.java   |   20 +-
 ...CommandOptionHasMultipleValuesException.java |   20 +-
 .../CliCommandOptionInvalidException.java       |   20 +-
 .../CliCommandOptionMissingException.java       |   20 +-
 .../CliCommandOptionNotApplicableException.java |   20 +-
 ...liCommandOptionValueConversionException.java |   20 +-
 .../CliCommandOptionValueException.java         |   20 +-
 .../CliCommandOptionValueMissingException.java  |   20 +-
 .../internal/cli/exceptions/CliException.java   |   20 +-
 .../exceptions/CreateSubregionException.java    |   20 +-
 .../cli/exceptions/ExceptionGenerator.java      |   20 +-
 .../cli/exceptions/ExceptionHandler.java        |   20 +-
 .../cli/exceptions/IndexNotFoundException.java  |   16 +
 .../cli/functions/AlterHDFSStoreFunction.java   |  234 -
 .../functions/AlterRuntimeConfigFunction.java   |   21 +-
 .../cli/functions/ChangeLogLevelFunction.java   |   20 +-
 .../cli/functions/CliFunctionResult.java        |   21 +-
 .../functions/CloseDurableClientFunction.java   |   21 +-
 .../cli/functions/CloseDurableCqFunction.java   |   21 +-
 .../cli/functions/ContunuousQueryFunction.java  |   24 +-
 .../CreateAsyncEventQueueFunction.java          |   29 +-
 .../functions/CreateDefinedIndexesFunction.java |   16 +
 .../cli/functions/CreateDiskStoreFunction.java  |   21 +-
 .../cli/functions/CreateHDFSStoreFunction.java  |  112 -
 .../cli/functions/CreateIndexFunction.java      |   21 +-
 .../cli/functions/DataCommandFunction.java      |   27 +-
 .../internal/cli/functions/DeployFunction.java  |   21 +-
 .../functions/DescribeDiskStoreFunction.java    |   18 +-
 .../functions/DescribeHDFSStoreFunction.java    |   16 +
 .../cli/functions/DestroyDiskStoreFunction.java |   21 +-
 .../cli/functions/DestroyHDFSStoreFunction.java |  100 -
 .../cli/functions/DestroyIndexFunction.java     |   21 +-
 .../cli/functions/ExportConfigFunction.java     |   23 +-
 .../cli/functions/ExportDataFunction.java       |   21 +-
 .../ExportSharedConfigurationFunction.java      |   21 +-
 .../FetchRegionAttributesFunction.java          |   20 +-
 .../FetchSharedConfigurationStatusFunction.java |   21 +-
 .../functions/GarbageCollectionFunction.java    |   21 +-
 .../GatewayReceiverCreateFunction.java          |   21 +-
 .../functions/GatewayReceiverFunctionArgs.java  |   21 +-
 .../functions/GatewaySenderCreateFunction.java  |   21 +-
 .../functions/GatewaySenderFunctionArgs.java    |   21 +-
 .../GetMemberConfigInformationFunction.java     |   21 +-
 .../functions/GetMemberInformationFunction.java |   25 +-
 .../functions/GetRegionDescriptionFunction.java |   20 +-
 .../cli/functions/GetRegionsFunction.java       |   20 +-
 .../cli/functions/GetStackTracesFunction.java   |   21 +-
 .../GetSubscriptionQueueSizeFunction.java       |   21 +-
 .../cli/functions/ImportDataFunction.java       |   21 +-
 ...ortSharedConfigurationArtifactsFunction.java |   21 +-
 .../functions/ListAsyncEventQueuesFunction.java |   18 +-
 .../cli/functions/ListDeployedFunction.java     |   21 +-
 .../cli/functions/ListDiskStoresFunction.java   |   18 +-
 .../functions/ListDurableCqNamesFunction.java   |   20 +-
 .../cli/functions/ListFunctionFunction.java     |   21 +-
 .../cli/functions/ListHDFSStoresFunction.java   |  102 -
 .../cli/functions/ListIndexFunction.java        |   20 +-
 .../LoadSharedConfigurationFunction.java        |   21 +-
 .../internal/cli/functions/LogFileFunction.java |   23 +-
 .../cli/functions/MemberRegionFunction.java     |   23 +-
 .../cli/functions/MembersForRegionFunction.java |   20 +-
 .../internal/cli/functions/NetstatFunction.java |   20 +-
 .../cli/functions/RebalanceFunction.java        |   23 +-
 .../cli/functions/RegionAlterFunction.java      |   20 +-
 .../cli/functions/RegionCreateFunction.java     |   33 +-
 .../cli/functions/RegionDestroyFunction.java    |   20 +-
 .../cli/functions/RegionFunctionArgs.java       |  109 +-
 .../cli/functions/ShutDownFunction.java         |   23 +-
 .../cli/functions/UndeployFunction.java         |   21 +-
 .../cli/functions/UnregisterFunction.java       |   23 +-
 .../cli/functions/UserFunctionExecution.java    |   24 +-
 .../management/internal/cli/help/CliTopic.java  |   20 +-
 .../internal/cli/help/format/Block.java         |   20 +-
 .../internal/cli/help/format/DataNode.java      |   20 +-
 .../internal/cli/help/format/Help.java          |   20 +-
 .../internal/cli/help/format/NewHelp.java       |   20 +-
 .../internal/cli/help/format/Row.java           |   20 +-
 .../internal/cli/help/utils/FormatOutput.java   |   16 +
 .../internal/cli/help/utils/HelpUtils.java      |   20 +-
 .../internal/cli/i18n/CliStrings.java           |  201 +-
 .../internal/cli/json/GfJsonArray.java          |   20 +-
 .../internal/cli/json/GfJsonException.java      |   22 +-
 .../internal/cli/json/GfJsonObject.java         |   20 +-
 .../management/internal/cli/json/TypedJson.java |   20 +-
 .../internal/cli/modes/CommandModes.java        |   18 +-
 .../cli/multistep/CLIMultiStepHelper.java       |   27 +-
 .../internal/cli/multistep/CLIRemoteStep.java   |   16 +
 .../internal/cli/multistep/CLIStep.java         |   16 +
 .../cli/multistep/CLIStepExecption.java         |   21 +-
 .../cli/multistep/MultiStepCommand.java         |   16 +
 .../internal/cli/parser/Argument.java           |   20 +-
 .../internal/cli/parser/AvailabilityTarget.java |   20 +-
 .../internal/cli/parser/CommandTarget.java      |   22 +-
 .../internal/cli/parser/GfshMethodTarget.java   |   30 +-
 .../internal/cli/parser/GfshOptionParser.java   |   22 +-
 .../internal/cli/parser/MethodParameter.java    |   20 +-
 .../management/internal/cli/parser/Option.java  |   20 +-
 .../internal/cli/parser/OptionSet.java          |   20 +-
 .../internal/cli/parser/Parameter.java          |   20 +-
 .../internal/cli/parser/ParserUtils.java        |   20 +-
 .../internal/cli/parser/SyntaxConstants.java    |   22 +-
 .../cli/parser/jopt/JoptOptionParser.java       |   40 +-
 .../preprocessor/EnclosingCharacters.java       |   20 +-
 .../cli/parser/preprocessor/Preprocessor.java   |   20 +-
 .../parser/preprocessor/PreprocessorUtils.java  |   31 +-
 .../internal/cli/parser/preprocessor/Stack.java |   20 +-
 .../cli/parser/preprocessor/TrimmedInput.java   |   20 +-
 .../cli/remote/CommandExecutionContext.java     |   20 +-
 .../internal/cli/remote/CommandProcessor.java   |   20 +-
 .../cli/remote/CommandStatementImpl.java        |   20 +-
 .../cli/remote/MemberCommandService.java        |   20 +-
 .../cli/remote/RemoteExecutionStrategy.java     |   24 +-
 .../internal/cli/remote/WrapperThreadLocal.java |   20 +-
 .../internal/cli/result/AbstractResultData.java |   20 +-
 .../cli/result/CliJsonSerializable.java         |   20 +-
 .../cli/result/CliJsonSerializableFactory.java  |   21 +-
 .../cli/result/CliJsonSerializableIds.java      |   20 +-
 .../internal/cli/result/CommandResult.java      |   20 +-
 .../cli/result/CommandResultException.java      |   21 +-
 .../cli/result/CompositeResultData.java         |   20 +-
 .../internal/cli/result/ErrorResultData.java    |   20 +-
 .../internal/cli/result/FileResult.java         |   20 +-
 .../internal/cli/result/InfoResultData.java     |   20 +-
 .../internal/cli/result/ObjectResultData.java   |   20 +-
 .../internal/cli/result/ResultBuilder.java      |   20 +-
 .../internal/cli/result/ResultData.java         |   20 +-
 .../cli/result/ResultDataException.java         |   20 +-
 .../internal/cli/result/TableBuilder.java       |   20 +-
 .../internal/cli/result/TableBuilderHelper.java |   21 +-
 .../internal/cli/result/TabularResultData.java  |   20 +-
 .../management/internal/cli/shell/Gfsh.java     |   20 +-
 .../internal/cli/shell/GfshConfig.java          |   20 +-
 .../cli/shell/GfshExecutionStrategy.java        |   24 +-
 .../cli/shell/JMXConnectionException.java       |   20 +-
 .../cli/shell/JMXInvocationException.java       |   20 +-
 .../internal/cli/shell/JmxOperationInvoker.java |   20 +-
 .../internal/cli/shell/MultiCommandHelper.java  |   16 +
 .../internal/cli/shell/OperationInvoker.java    |   20 +-
 .../internal/cli/shell/jline/ANSIHandler.java   |   20 +-
 .../cli/shell/jline/CygwinMinttyTerminal.java   |   21 +-
 .../internal/cli/shell/jline/GfshHistory.java   |   20 +-
 .../shell/jline/GfshUnsupportedTerminal.java    |   20 +-
 .../cli/shell/unsafe/GfshSignalHandler.java     |   21 +-
 .../internal/cli/util/CLIConsoleBufferUtil.java |   21 +-
 .../internal/cli/util/CauseFinder.java          |   20 +-
 .../cli/util/ClasspathScanLoadHelper.java       |   20 +-
 .../internal/cli/util/CommandStringBuilder.java |   20 +-
 .../internal/cli/util/CommentSkipHelper.java    |   22 +-
 .../internal/cli/util/ConnectionEndpoint.java   |   21 +-
 .../internal/cli/util/DiskStoreCompacter.java   |   20 +-
 .../cli/util/DiskStoreNotFoundException.java    |   18 +-
 .../internal/cli/util/DiskStoreUpgrader.java    |   21 +-
 .../internal/cli/util/DiskStoreValidater.java   |   21 +-
 .../cli/util/EvictionAttributesInfo.java        |   21 +-
 .../cli/util/FixedPartitionAttributesInfo.java  |   21 +-
 .../internal/cli/util/GfshConsoleReader.java    |   22 +-
 .../cli/util/HDFSStoreNotFoundException.java    |   18 +-
 .../cli/util/JConsoleNotFoundException.java     |   20 +-
 .../management/internal/cli/util/JsonUtil.java  |   20 +-
 .../internal/cli/util/MemberInformation.java    |   20 +-
 .../cli/util/MemberNotFoundException.java       |   18 +-
 .../management/internal/cli/util/MergeLogs.java |   21 +-
 .../internal/cli/util/ReadWriteFile.java        |   21 +-
 .../cli/util/RegionAttributesDefault.java       |   21 +-
 .../cli/util/RegionAttributesNames.java         |   21 +-
 .../internal/cli/util/RegionPath.java           |   20 +-
 .../cli/util/VisualVmNotFoundException.java     |   20 +-
 .../internal/cli/util/spring/Assert.java        |   27 -
 .../internal/cli/util/spring/ObjectUtils.java   |  291 --
 .../cli/util/spring/ReflectionUtils.java        |  123 -
 .../internal/cli/util/spring/StringUtils.java   |  220 -
 .../SharedConfigurationWriter.java              |   21 +-
 .../callbacks/ConfigurationChangeListener.java  |   21 +-
 .../configuration/domain/CacheElement.java      |   21 +-
 .../configuration/domain/Configuration.java     |   21 +-
 .../domain/ConfigurationChangeResult.java       |   21 +-
 .../domain/SharedConfigurationStatus.java       |   16 +
 .../configuration/domain/XmlEntity.java         |   21 +-
 .../configuration/functions/AddJarFunction.java |   21 +-
 .../functions/AddXmlEntityFunction.java         |   21 +-
 .../functions/DeleteJarFunction.java            |   21 +-
 .../functions/DeleteXmlEntityFunction.java      |   21 +-
 .../functions/GetAllJarsFunction.java           |   21 +-
 .../functions/ModifyPropertiesFunction.java     |   21 +-
 .../handlers/ConfigurationRequestHandler.java   |   21 +-
 ...SharedConfigurationStatusRequestHandler.java |   21 +-
 .../messages/ConfigurationRequest.java          |   21 +-
 .../messages/ConfigurationResponse.java         |   21 +-
 .../SharedConfigurationStatusRequest.java       |   21 +-
 .../SharedConfigurationStatusResponse.java      |   21 +-
 .../configuration/utils/DtdResolver.java        |   16 +
 .../configuration/utils/XmlConstants.java       |   21 +-
 .../internal/configuration/utils/XmlUtils.java  |   21 +-
 .../internal/configuration/utils/ZipUtils.java  |   21 +-
 .../internal/messages/CompactRequest.java       |   20 +-
 .../internal/messages/CompactResponse.java      |   20 +-
 .../internal/security/AccessControl.java        |   16 +
 .../internal/security/AccessControlContext.java |   16 +
 .../internal/security/AccessControlMXBean.java  |   16 +
 .../internal/security/CLIOperationContext.java  |   16 +
 .../internal/security/JMXOperationContext.java  |   16 +
 .../internal/security/JSONAuthorization.java    |   16 +
 .../internal/security/MBeanServerWrapper.java   |   16 +
 .../security/ManagementInterceptor.java         |   16 +
 .../management/internal/security/Resource.java  |   16 +
 .../internal/security/ResourceConstants.java    |   16 +
 .../internal/security/ResourceOperation.java    |   16 +
 .../security/ResourceOperationContext.java      |   16 +
 .../unsafe/ReadOpFileAccessController.java      |   21 +-
 .../controllers/AbstractCommandsController.java |   28 +-
 .../AbstractMultiPartCommandsController.java    |   21 +-
 .../controllers/ClientCommandsController.java   |   21 +-
 .../controllers/ClusterCommandsController.java  |   21 +-
 .../controllers/ConfigCommandsController.java   |   21 +-
 .../web/controllers/DataCommandsController.java |   21 +-
 .../controllers/DeployCommandsController.java   |   21 +-
 .../DiskStoreCommandsController.java            |   21 +-
 .../DurableClientCommandsController.java        |   21 +-
 .../controllers/FunctionCommandsController.java |   21 +-
 .../HDFSStoreCommandsController.java            |  229 -
 .../controllers/IndexCommandsController.java    |   21 +-
 .../LauncherLifecycleCommandsController.java    |   21 +-
 .../controllers/MemberCommandsController.java   |   21 +-
 .../MiscellaneousCommandsController.java        |   21 +-
 .../web/controllers/PdxCommandsController.java  |   16 +
 .../controllers/QueueCommandsController.java    |   27 +-
 .../controllers/RegionCommandsController.java   |   21 +-
 .../controllers/ShellCommandsController.java    |  225 +-
 .../web/controllers/WanCommandsController.java  |   23 +-
 .../EnvironmentVariablesHandlerInterceptor.java |   21 +-
 .../support/MemberMXBeanAdapter.java            |   21 +-
 .../management/internal/web/domain/Link.java    |   21 +-
 .../internal/web/domain/LinkIndex.java          |   21 +-
 .../web/domain/QueryParameterSource.java        |   21 +-
 .../internal/web/http/ClientHttpRequest.java    |   21 +-
 .../internal/web/http/HttpHeader.java           |   21 +-
 .../internal/web/http/HttpMethod.java           |   21 +-
 .../SerializableObjectHttpMessageConverter.java |   21 +-
 .../web/http/support/SimpleHttpRequester.java   |   21 +-
 .../internal/web/io/MultipartFileAdapter.java   |   21 +-
 .../web/io/MultipartFileResourceAdapter.java    |   21 +-
 .../web/shell/AbstractHttpOperationInvoker.java |   21 +-
 .../web/shell/HttpOperationInvoker.java         |   16 +
 .../web/shell/MBeanAccessException.java         |   21 +-
 .../RestApiCallForCommandNotFoundException.java |   21 +-
 .../web/shell/RestHttpOperationInvoker.java     |   21 +-
 .../web/shell/SimpleHttpOperationInvoker.java   |   21 +-
 .../shell/support/HttpInvocationHandler.java    |   21 +-
 .../shell/support/HttpMBeanProxyFactory.java    |   21 +-
 .../internal/web/util/ConvertUtils.java         |   21 +-
 .../management/internal/web/util/UriUtils.java  |   21 +-
 .../management/membership/ClientMembership.java |   29 +-
 .../membership/ClientMembershipEvent.java       |   21 +-
 .../membership/ClientMembershipListener.java    |   21 +-
 .../ClientMembershipListenerAdapter.java        |   21 +-
 .../management/membership/MembershipEvent.java  |   21 +-
 .../membership/MembershipListener.java          |   21 +-
 .../UniversalMembershipListenerAdapter.java     |   21 +-
 .../gemstone/gemfire/management/package.html    |   16 +
 .../memcached/GemFireMemcachedServer.java       |   21 +-
 .../main/java/com/gemstone/gemfire/package.html |   16 +
 .../com/gemstone/gemfire/pdx/FieldType.java     |   21 +-
 .../com/gemstone/gemfire/pdx/JSONFormatter.java |   16 +
 .../gemfire/pdx/JSONFormatterException.java     |   23 +-
 .../gemfire/pdx/NonPortableClassException.java  |   16 +
 .../gemfire/pdx/PdxConfigurationException.java  |   21 +-
 .../pdx/PdxFieldAlreadyExistsException.java     |   21 +-
 .../pdx/PdxFieldDoesNotExistException.java      |   21 +-
 .../pdx/PdxFieldTypeMismatchException.java      |   21 +-
 .../gemfire/pdx/PdxInitializationException.java |   21 +-
 .../com/gemstone/gemfire/pdx/PdxInstance.java   |   21 +-
 .../gemfire/pdx/PdxInstanceFactory.java         |   21 +-
 .../com/gemstone/gemfire/pdx/PdxReader.java     |   21 +-
 .../pdx/PdxRegistryMismatchException.java       |   24 +-
 .../gemstone/gemfire/pdx/PdxSerializable.java   |   21 +-
 .../gemfire/pdx/PdxSerializationException.java  |   21 +-
 .../com/gemstone/gemfire/pdx/PdxSerializer.java |   21 +-
 .../gemstone/gemfire/pdx/PdxUnreadFields.java   |   21 +-
 .../com/gemstone/gemfire/pdx/PdxWriter.java     |   21 +-
 .../pdx/ReflectionBasedAutoSerializer.java      |   30 +-
 .../gemfire/pdx/WritablePdxInstance.java        |   21 +-
 .../pdx/internal/AutoSerializableManager.java   |   29 +-
 .../pdx/internal/CheckTypeRegistryState.java    |   21 +-
 .../pdx/internal/ClientTypeRegistration.java    |   21 +-
 .../gemfire/pdx/internal/ComparableEnum.java    |   16 +
 .../pdx/internal/ConvertableToBytes.java        |   16 +
 .../gemstone/gemfire/pdx/internal/DataSize.java |   21 +-
 .../gemfire/pdx/internal/DefaultPdxField.java   |   21 +-
 .../gemstone/gemfire/pdx/internal/EnumId.java   |   21 +-
 .../gemstone/gemfire/pdx/internal/EnumInfo.java |   21 +-
 .../pdx/internal/FieldNotFoundInPdxVersion.java |   16 +
 .../gemfire/pdx/internal/InternalPdxReader.java |   21 +-
 .../pdx/internal/LonerTypeRegistration.java     |   21 +-
 .../pdx/internal/NullTypeRegistration.java      |   21 +-
 .../gemstone/gemfire/pdx/internal/PdxField.java |   21 +-
 .../gemfire/pdx/internal/PdxInputStream.java    |   23 +-
 .../gemfire/pdx/internal/PdxInstanceEnum.java   |   21 +-
 .../pdx/internal/PdxInstanceFactoryImpl.java    |   21 +-
 .../gemfire/pdx/internal/PdxInstanceImpl.java   |   21 +-
 .../pdx/internal/PdxInstanceInputStream.java    |   21 +-
 .../gemfire/pdx/internal/PdxOutputStream.java   |   21 +-
 .../gemfire/pdx/internal/PdxReaderImpl.java     |   21 +-
 .../gemfire/pdx/internal/PdxString.java         |   23 +-
 .../gemstone/gemfire/pdx/internal/PdxType.java  |   21 +-
 .../gemfire/pdx/internal/PdxUnreadData.java     |   21 +-
 .../gemfire/pdx/internal/PdxWriterImpl.java     |   21 +-
 .../pdx/internal/PeerTypeRegistration.java      |   21 +-
 .../pdx/internal/TrackingPdxReaderImpl.java     |   21 +-
 .../gemfire/pdx/internal/TypeRegistration.java  |   21 +-
 .../gemfire/pdx/internal/TypeRegistry.java      |   40 +-
 .../gemfire/pdx/internal/UnreadPdxType.java     |   21 +-
 .../internal/WeakConcurrentIdentityHashMap.java |   21 +-
 .../pdx/internal/WritablePdxInstanceImpl.java   |   21 +-
 .../gemfire/pdx/internal/json/JsonHelper.java   |   21 +-
 .../pdx/internal/json/PdxInstanceHelper.java    |   23 +-
 .../pdx/internal/json/PdxListHelper.java        |   23 +-
 .../gemfire/pdx/internal/json/PdxToJSON.java    |   23 +-
 .../pdx/internal/unsafe/UnsafeWrapper.java      |   21 +-
 .../java/com/gemstone/gemfire/pdx/package.html  |   15 +
 .../com/gemstone/gemfire/ra/GFConnection.java   |   16 +
 .../gemfire/ra/GFConnectionFactory.java         |   16 +
 .../gemfire/redis/GemFireRedisServer.java       |  173 +-
 .../gemfire/security/AccessControl.java         |   21 +-
 .../gemfire/security/AuthInitialize.java        |   21 +-
 .../security/AuthenticationFailedException.java |   21 +-
 .../AuthenticationRequiredException.java        |   21 +-
 .../gemfire/security/Authenticator.java         |   21 +-
 .../security/GemFireSecurityException.java      |   21 +-
 .../security/NotAuthorizedException.java        |   21 +-
 .../com/gemstone/gemfire/security/package.html  |   16 +
 .../util/concurrent/SynchronousQueueNoSpin.java | 1144 -----
 .../src/main/java/external-overview.html        |   16 +
 .../src/main/java/internal-overview.html        |   16 +
 .../gemfire/cache/cache-8.1.xsd                 |   16 +
 .../gemfire/cache/cache-9.0.xsd                 |   42 +-
 .../services/org.xml.sax.ext.EntityResolver2    |    1 +
 .../gemstone/gemfire/admin/doc-files/ds4_0.dtd  |   15 +
 .../gemstone/gemfire/admin/doc-files/ds5_0.dtd  |   15 +
 .../gemfire/cache/doc-files/cache3_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache4_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache4_1.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_1.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_5.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_7.dtd        |   15 +
 .../gemfire/cache/doc-files/cache5_8.dtd        |   15 +
 .../gemfire/cache/doc-files/cache6_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache6_1.dtd        |   15 +
 .../gemfire/cache/doc-files/cache6_5.dtd        |   15 +
 .../gemfire/cache/doc-files/cache6_6.dtd        |   15 +
 .../gemfire/cache/doc-files/cache7_0.dtd        |   15 +
 .../gemfire/cache/doc-files/cache8_0.dtd        |   15 +
 .../distributed/internal/javagroups-config.txt  |  101 -
 .../distributed/internal/javagroups-mcast.txt   |   97 -
 .../membership/gms/messenger/jgroups-config.xml |   72 +
 .../membership/gms/messenger/jgroups-mcast.xml  |   98 +
 .../internal/logging/log4j/log4j2-cli.xml       |   17 -
 .../internal/logging/log4j/log4j2-default.xml   |   21 -
 .../tools/gfsh/app/windowsbindings.properties   |   15 +
 .../internal/cli/commands/support/gfmon.html    |   20 +-
 .../com/gemstone/gemfire/statisticsType.dtd     |   17 +-
 gemfire-core/src/main/resources/log4j2-cli.xml  |   17 +
 gemfire-core/src/main/resources/log4j2.xml      |   23 +
 .../batterytest/greplogs/ExpectedStrings.java   |   25 +-
 .../java/batterytest/greplogs/LogConsumer.java  |   66 +-
 .../src/test/java/cacheRunner/Portfolio.java    |   16 +
 .../src/test/java/cacheRunner/Position.java     |   16 +
 .../src/test/java/com/company/app/Customer.java |   21 +-
 .../src/test/java/com/company/app/DBLoader.java |   21 +-
 .../com/company/app/OrdersCacheListener.java    |   21 +-
 .../java/com/company/data/DatabaseLoader.java   |   21 +-
 .../java/com/company/data/MyDeclarable.java     |   16 +
 .../src/test/java/com/company/data/MySizer.java |   21 +-
 .../com/company/data/MyTransactionListener.java |   21 +-
 .../src/test/java/com/examples/LinkNode.java    |   21 +-
 .../src/test/java/com/examples/SuperClass.java  |   21 +-
 .../src/test/java/com/examples/TestObject.java  |   21 +-
 .../src/test/java/com/examples/ds/Address.java  |   16 +
 .../src/test/java/com/examples/ds/Company.java  |   21 +-
 .../java/com/examples/ds/CompanySerializer.java |   21 +-
 .../src/test/java/com/examples/ds/Employee.java |   21 +-
 .../com/examples/ds/PutDataSerializables.java   |   21 +-
 .../src/test/java/com/examples/ds/User.java     |   21 +-
 .../com/examples/snapshot/MyDataSerializer.java |   21 +-
 .../java/com/examples/snapshot/MyObject.java    |   23 +-
 .../snapshot/MyObjectDataSerializable.java      |   23 +-
 .../java/com/examples/snapshot/MyObjectPdx.java |   16 +
 .../snapshot/MyObjectPdxSerializable.java       |   21 +-
 .../com/examples/snapshot/MyPdxSerializer.java  |   21 +-
 .../java/com/gemstone/gemfire/AppObject.java    |   18 +-
 .../test/java/com/gemstone/gemfire/BadTest.java |   21 +-
 .../com/gemstone/gemfire/CopyJUnitTest.java     |   21 +-
 .../com/gemstone/gemfire/DeltaTestImpl.java     |   21 +-
 .../gemfire/DiskInstantiatorsJUnitTest.java     |   21 +-
 .../com/gemstone/gemfire/GemFireTestCase.java   |   21 +-
 .../java/com/gemstone/gemfire/Invariant.java    |   22 +-
 .../com/gemstone/gemfire/InvariantResult.java   |   22 +-
 .../com/gemstone/gemfire/JUnitTestSetup.java    |   21 +-
 .../gemfire/JtaNoninvolvementJUnitTest.java     |   21 +-
 .../gemfire/LocalStatisticsJUnitTest.java       |   21 +-
 .../com/gemstone/gemfire/LonerDMJUnitTest.java  |   21 +-
 .../gemstone/gemfire/StatisticsTestCase.java    |   21 +-
 .../gemfire/StatisticsTypeJUnitTest.java        |   21 +-
 .../com/gemstone/gemfire/TXExpiryJUnitTest.java |  569 ++-
 .../java/com/gemstone/gemfire/TXJUnitTest.java  |   21 +-
 .../com/gemstone/gemfire/TXWriterJUnitTest.java |   21 +-
 .../gemstone/gemfire/TXWriterOOMEJUnitTest.java |   21 +-
 .../com/gemstone/gemfire/TXWriterTestCase.java  |   16 +
 .../gemstone/gemfire/TestDataSerializer.java    |   21 +-
 .../com/gemstone/gemfire/TimingTestCase.java    |   22 +-
 .../com/gemstone/gemfire/UnitTestDoclet.java    |   21 +-
 .../gemstone/gemfire/admin/AdminTestHelper.java |   16 +
 .../BindDistributedSystemJUnitTest.java         |  177 +-
 .../internal/CacheHealthEvaluatorJUnitTest.java |   21 +-
 .../internal/DistributedSystemTestCase.java     |   21 +-
 .../admin/internal/HealthEvaluatorTestCase.java |   21 +-
 .../MemberHealthEvaluatorJUnitTest.java         |   21 +-
 .../cache/AttributesFactoryJUnitTest.java       |   39 +-
 .../gemfire/cache/Bug36619JUnitTest.java        |   21 +-
 .../gemfire/cache/Bug42039JUnitTest.java        |   44 +-
 .../gemfire/cache/Bug52289JUnitTest.java        |   89 +
 .../gemfire/cache/CacheListenerJUnitTest.java   |   21 +-
 .../cache/CacheRegionClearStatsDUnitTest.java   |   21 +-
 .../gemstone/gemfire/cache/ClientHelper.java    |   22 +-
 .../cache/ClientServerTimeSyncDUnitTest.java    |   25 +-
 .../cache/ConnectionPoolAndLoaderDUnitTest.java |   25 +-
 .../cache/ConnectionPoolFactoryJUnitTest.java   |   21 +-
 .../gemfire/cache/OperationJUnitTest.java       |   21 +-
 .../gemfire/cache/PoolManagerJUnitTest.java     |   21 +-
 .../gemstone/gemfire/cache/ProxyJUnitTest.java  |   21 +-
 .../gemfire/cache/RegionFactoryJUnitTest.java   |   40 +-
 .../gemfire/cache/RoleExceptionJUnitTest.java   |   21 +-
 .../SerialAsyncEventQueueImplJUnitTest.java     |   62 +
 .../client/ClientCacheFactoryJUnitTest.java     |   71 +-
 .../client/ClientRegionFactoryJUnitTest.java    |   21 +-
 .../ClientServerRegisterInterestsDUnitTest.java |   21 +-
 .../internal/AutoConnectionSourceDUnitTest.java |   50 +-
 .../AutoConnectionSourceImplJUnitTest.java      |   21 +-
 .../AutoConnectionSourceWithUDPDUnitTest.java   |  140 -
 .../internal/CacheServerSSLConnectionDUnit.java |  648 ---
 .../CacheServerSSLConnectionDUnitTest.java      |  426 ++
 .../internal/ConnectionPoolImplJUnitTest.java   |   32 +-
 .../internal/LocatorLoadBalancingDUnitTest.java |   31 +-
 .../cache/client/internal/LocatorTestBase.java  |   33 +-
 .../internal/OpExecutorImplJUnitTest.java       |   21 +-
 .../client/internal/QueueManagerJUnitTest.java  |   21 +-
 .../internal/SSLNoClientAuthDUnitTest.java      |  280 ++
 .../internal/ServerBlackListJUnitTest.java      |   21 +-
 .../locator/LocatorStatusResponseJUnitTest.java |   20 +-
 .../pooling/ConnectionManagerJUnitTest.java     |   21 +-
 .../ColocatedRegionWithHDFSDUnitTest.java       |  193 -
 .../hdfs/internal/HDFSConfigJUnitTest.java      |  536 --
 .../hdfs/internal/HDFSEntriesSetJUnitTest.java  |  227 -
 .../internal/HdfsStoreMutatorJUnitTest.java     |  200 -
 .../hdfs/internal/RegionWithHDFSTestBase.java   |  687 ---
 .../SignalledFlushObserverJUnitTest.java        |   23 +-
 .../SortedListForAsyncQueueJUnitTest.java       |   31 +-
 .../internal/hoplog/BaseHoplogTestCase.java     |  392 --
 .../hoplog/CardinalityEstimatorJUnitTest.java   |  188 -
 .../hoplog/HDFSCacheLoaderJUnitTest.java        |  106 -
 .../hoplog/HDFSCompactionManagerJUnitTest.java  |  449 --
 .../hoplog/HDFSRegionDirectorJUnitTest.java     |   97 -
 .../internal/hoplog/HDFSStatsJUnitTest.java     |  250 -
 .../HDFSUnsortedHoplogOrganizerJUnitTest.java   |  297 --
 .../HdfsSortedOplogOrganizerJUnitTest.java      | 1045 ----
 .../hoplog/HfileSortedOplogJUnitTest.java       |  540 --
 .../hoplog/SortedOplogListIterJUnitTest.java    |  178 -
 .../hoplog/TieredCompactionJUnitTest.java       |  907 ----
 .../hoplog/mapreduce/GFKeyJUnitTest.java        |   50 -
 .../mapreduce/HDFSSplitIteratorJUnitTest.java   |  265 -
 .../hoplog/mapreduce/HoplogUtilJUnitTest.java   |  308 --
 .../management/MXMemoryPoolListenerExample.java |   21 +-
 .../management/MemoryThresholdsDUnitTest.java   |  186 +-
 .../MemoryThresholdsOffHeapDUnitTest.java       |  347 +-
 .../management/ResourceManagerDUnitTest.java    |   21 +-
 .../ExceptionHandlingJUnitTest.java             |   21 +-
 .../mapInterface/MapFunctionalJUnitTest.java    |   21 +-
 .../mapInterface/PutAllGlobalLockJUnitTest.java |   21 +-
 .../PutOperationContextJUnitTest.java           |  264 +
 .../GetOperationContextImplJUnitTest.java       |  291 ++
 .../partition/PartitionManagerDUnitTest.java    |  509 --
 .../PartitionRegionHelperDUnitTest.java         |   21 +-
 .../BaseLineAndCompareQueryPerfJUnitTest.java   |   21 +-
 .../query/Bug32947ValueConstraintJUnitTest.java |   21 +-
 .../gemfire/cache/query/BugJUnitTest.java       |   21 +-
 .../gemfire/cache/query/CacheUtils.java         |   21 +-
 .../cache/query/PdxStringQueryJUnitTest.java    |   21 +-
 .../gemstone/gemfire/cache/query/PerfQuery.java |   22 +-
 .../gemfire/cache/query/QueryJUnitTest.java     |  112 +-
 .../cache/query/QueryServiceJUnitTest.java      |   21 +-
 .../gemfire/cache/query/QueryTestUtils.java     |   21 +-
 .../cache/query/QueryTestUtilsJUnitTest.java    |   21 +-
 .../gemfire/cache/query/RegionJUnitTest.java    |   21 +-
 .../cache/query/TypedIteratorJUnitTest.java     |   21 +-
 .../com/gemstone/gemfire/cache/query/Utils.java |   21 +-
 .../query/cq/dunit/CqQueryTestListener.java     |   21 +-
 .../gemfire/cache/query/data/Address.java       |   21 +-
 .../gemstone/gemfire/cache/query/data/City.java |   22 +-
 .../cache/query/data/CollectionHolder.java      |   22 +-
 .../cache/query/data/ComparableWrapper.java     |   22 +-
 .../gemfire/cache/query/data/Country.java       |   21 +-
 .../gemstone/gemfire/cache/query/data/Data.java |   22 +-
 .../gemfire/cache/query/data/District.java      |   22 +-
 .../gemfire/cache/query/data/Employee.java      |   21 +-
 .../gemfire/cache/query/data/Inventory.java     |   21 +-
 .../gemfire/cache/query/data/Keywords.java      |   21 +-
 .../gemfire/cache/query/data/Manager.java       |   21 +-
 .../gemfire/cache/query/data/Numbers.java       |   21 +-
 .../gemfire/cache/query/data/PhoneNo.java       |   21 +-
 .../gemfire/cache/query/data/Portfolio.java     |   22 +-
 .../gemfire/cache/query/data/PortfolioData.java |   21 +-
 .../gemfire/cache/query/data/PortfolioNoDS.java |   16 +
 .../gemfire/cache/query/data/PortfolioPdx.java  |   25 +-
 .../gemfire/cache/query/data/Position.java      |   21 +-
 .../gemfire/cache/query/data/PositionNoDS.java  |   16 +
 .../gemfire/cache/query/data/PositionPdx.java   |   21 +-
 .../query/data/ProhibitedSecurityQuote.java     |   21 +-
 .../gemfire/cache/query/data/Quote.java         |   21 +-
 .../gemfire/cache/query/data/Restricted.java    |   21 +-
 .../cache/query/data/SecurityMaster.java        |   21 +-
 .../gemfire/cache/query/data/State.java         |   21 +-
 .../gemfire/cache/query/data/Street.java        |   21 +-
 .../gemfire/cache/query/data/Student.java       |   23 +-
 .../gemfire/cache/query/data/Vehicle.java       |   21 +-
 .../gemfire/cache/query/data/Village.java       |   21 +-
 .../query/dunit/CloseCacheAuthorization.java    |   18 +-
 .../query/dunit/CompactRangeIndexDUnitTest.java |   21 +-
 .../cache/query/dunit/CqTimeTestListener.java   |   21 +-
 .../cache/query/dunit/GroupByDUnitImpl.java     |   16 +
 .../dunit/GroupByPartitionedQueryDUnitTest.java |   16 +
 .../query/dunit/GroupByQueryDUnitTest.java      |   16 +
 .../cache/query/dunit/HashIndexDUnitTest.java   |   21 +-
 .../cache/query/dunit/HelperTestCase.java       |   16 +
 .../dunit/NonDistinctOrderByDUnitImpl.java      |   16 +
 .../NonDistinctOrderByPartitionedDUnitTest.java |   16 +
 .../query/dunit/PdxStringQueryDUnitTest.java    |   37 +-
 .../dunit/QueryAPITestPartitionResolver.java    |   22 +-
 .../cache/query/dunit/QueryAuthorization.java   |   21 +-
 .../dunit/QueryDataInconsistencyDUnitTest.java  |   24 +-
 .../dunit/QueryIndexUsingXMLDUnitTest.java      |   21 +-
 .../QueryParamsAuthorizationDUnitTest.java      |   21 +-
 .../QueryUsingFunctionContextDUnitTest.java     |   47 +-
 .../query/dunit/QueryUsingPoolDUnitTest.java    |   63 +-
 .../cache/query/dunit/RemoteQueryDUnitTest.java |  138 +-
 ...esourceManagerWithQueryMonitorDUnitTest.java |   62 +-
 .../query/dunit/SelectStarQueryDUnitTest.java   |   21 +-
 .../cache/query/facets/lang/Address.java        |   23 +-
 .../gemfire/cache/query/facets/lang/Course.java |   23 +-
 .../cache/query/facets/lang/Department.java     |   23 +-
 .../query/facets/lang/DerivedEmployee.java      |   22 +-
 .../cache/query/facets/lang/Employee.java       |   21 +-
 .../cache/query/facets/lang/Faculty.java        |   21 +-
 .../cache/query/facets/lang/G_Student.java      |   23 +-
 .../gemfire/cache/query/facets/lang/Person.java |   23 +-
 .../cache/query/facets/lang/Student.java        |   23 +-
 .../cache/query/facets/lang/UG_Student.java     |   23 +-
 .../gemfire/cache/query/facets/lang/Utils.java  |   20 +-
 .../ComparisonOperatorsJUnitTest.java           |   21 +-
 .../query/functional/ConstantsJUnitTest.java    |   21 +-
 .../query/functional/CountStarJUnitTest.java    |   21 +-
 .../CustomerOptimizationsJUnitTest.java         |   21 +-
 .../DistinctAndNonDistinctQueryJUnitTest.java   |   29 +-
 ...ctResultsWithDupValuesInRegionJUnitTest.java |   21 +-
 .../query/functional/FunctionJUnitTest.java     |   21 +-
 .../functional/GroupByPartitionedJUnitTest.java |   16 +
 .../functional/GroupByReplicatedJUnitTest.java  |   16 +
 .../cache/query/functional/GroupByTestImpl.java |   23 +-
 .../query/functional/GroupByTestInterface.java  |   16 +
 .../query/functional/INOperatorJUnitTest.java   |   21 +-
 .../functional/IUM6Bug32345ReJUnitTest.java     |   21 +-
 .../cache/query/functional/IUMJUnitTest.java    |   21 +-
 .../IUMRCompositeIteratorJUnitTest.java         |   21 +-
 .../IUMRMultiIndexesMultiRegionJUnitTest.java   |   21 +-
 .../IUMRShuffleIteratorsJUnitTest.java          |   21 +-
 .../functional/IUMRSingleRegionJUnitTest.java   |   21 +-
 ...ependentOperandsInWhereClause2JUnitTest.java |   21 +-
 .../IndexCreationDeadLockJUnitTest.java         |   21 +-
 .../functional/IndexCreationJUnitTest.java      |   23 +-
 .../IndexMaintenanceAsynchJUnitTest.java        |   21 +-
 .../functional/IndexOperatorJUnitTest.java      |   21 +-
 .../IndexPrimaryKeyUsageJUnitTest.java          |   21 +-
 .../IndexUsageInNestedQueryJUnitTest.java       |   21 +-
 .../IndexUsageWithAliasAsProjAtrbt.java         |   21 +-
 ...IndexUsageWithAliasAsProjAtrbtJUnitTest.java |   21 +-
 .../IndexUseMultFrmSnglCondJUnitTest.java       |   21 +-
 ...ndexWithSngleFrmAndMultCondQryJUnitTest.java |   21 +-
 .../functional/IteratorTypeDefEmpJUnitTest.java |   21 +-
 .../functional/IteratorTypeDefJUnitTest.java    |   21 +-
 .../IteratorTypeDefaultTypesJUnitTest.java      |   21 +-
 .../functional/IumMultConditionJUnitTest.java   |   21 +-
 .../functional/JavaSerializationJUnitTest.java  |   21 +-
 .../functional/LikePredicateJUnitTest.java      |   21 +-
 .../query/functional/LimitClauseJUnitTest.java  |   21 +-
 .../functional/LogicalOperatorsJUnitTest.java   |   21 +-
 .../cache/query/functional/MiscJUnitTest.java   |   21 +-
 .../functional/MultiIndexCreationJUnitTest.java |   16 +
 .../MultiRegionIndexUsageJUnitTest.java         |   21 +-
 .../functional/MultipleRegionsJUnitTest.java    |   21 +-
 .../NegativeNumberQueriesJUnitTest.java         |   21 +-
 .../query/functional/NestedQueryJUnitTest.java  |   21 +-
 .../NonDistinctOrderByPartitionedJUnitTest.java |   16 +
 .../NonDistinctOrderByReplicatedJUnitTest.java  |   16 +
 .../NonDistinctOrderByTestImplementation.java   |   21 +-
 .../query/functional/NumericQueryJUnitTest.java |   21 +-
 .../functional/OrderByPartitionedJUnitTest.java |   16 +
 .../functional/OrderByReplicatedJUnitTest.java  |   16 +
 .../functional/OrderByTestImplementation.java   |   21 +-
 .../functional/ParameterBindingJUnitTest.java   |   21 +-
 .../PdxGroupByPartitionedJUnitTest.java         |   16 +
 .../PdxGroupByReplicatedJUnitTest.java          |   16 +
 .../query/functional/PdxGroupByTestImpl.java    |   16 +
 .../query/functional/PdxOrderByJUnitTest.java   |   16 +
 .../functional/QRegionInterfaceJUnitTest.java   |   21 +-
 .../QueryREUpdateInProgressJUnitTest.java       |   21 +-
 .../functional/QueryUndefinedJUnitTest.java     |   21 +-
 .../functional/ReservedKeywordsJUnitTest.java   |   21 +-
 .../ResultsDataSerializabilityJUnitTest.java    |   21 +-
 .../query/functional/SelectToDateJUnitTest.java |   21 +-
 .../functional/StructMemberAccessJUnitTest.java |   21 +-
 .../query/functional/StructSetOrResultsSet.java |  281 +-
 .../query/functional/TestNewFunctionSSorRS.java |   21 +-
 .../CompiledAggregateFunctionJUnitTest.java     |   16 +
 .../CompiledGroupBySelectJUnitTest.java         |   16 +
 .../CompiledJunctionInternalsJUnitTest.java     |   21 +-
 .../internal/CopyOnReadQueryJUnitTest.java      |   21 +-
 .../internal/ExecutionContextJUnitTest.java     |   21 +-
 .../query/internal/IndexManagerJUnitTest.java   |   21 +-
 .../internal/NWayMergeResultsJUnitTest.java     |   16 +
 .../internal/OrderByComparatorJUnitTest.java    |   16 +
 .../internal/ProjectionAttributeJUnitTest.java  |   21 +-
 .../query/internal/QCompilerJUnitTest.java      |   21 +-
 ...ueryFromClauseCanonicalizationJUnitTest.java |   21 +-
 .../QueryObjectSerializationJUnitTest.java      |   21 +-
 .../QueryObserverCallbackJUnitTest.java         |   21 +-
 .../query/internal/QueryTraceJUnitTest.java     |   21 +-
 .../query/internal/QueryUtilsJUnitTest.java     |   21 +-
 .../query/internal/ResultsBagJUnitTest.java     |   21 +-
 .../ResultsBagLimitBehaviourJUnitTest.java      |   21 +-
 .../ResultsCollectionWrapperLimitJUnitTest.java |   21 +-
 .../SelectResultsComparatorJUnitTest.java       |   21 +-
 .../StructBagLimitBehaviourJUnitTest.java       |   21 +-
 .../query/internal/StructSetJUnitTest.java      |   21 +-
 .../internal/aggregate/AggregatorJUnitTest.java |   16 +
 ...syncIndexUpdaterThreadShutdownJUnitTest.java |   21 +-
 .../index/AsynchIndexMaintenanceJUnitTest.java  |   21 +-
 .../CompactRangeIndexIndexMapJUnitTest.java     |   21 +-
 .../index/CompactRangeIndexJUnitTest.java       |   21 +-
 ...rrentIndexInitOnOverflowRegionDUnitTest.java |   25 +-
 ...ndexOperationsOnOverflowRegionDUnitTest.java |   21 +-
 ...pdateWithInplaceObjectModFalseDUnitTest.java |   21 +-
 ...ConcurrentIndexUpdateWithoutWLDUnitTest.java |   21 +-
 .../index/CopyOnReadIndexDUnitTest.java         |  164 +-
 .../index/CopyOnReadIndexJUnitTest.java         |   26 +-
 .../DeclarativeIndexCreationJUnitTest.java      |   21 +-
 .../internal/index/HashIndexJUnitTest.java      |   62 +-
 .../internal/index/HashIndexSetJUnitTest.java   |  504 ++
 .../index/IndexCreationInternalsJUnitTest.java  |   21 +-
 .../internal/index/IndexElemArrayJUnitTest.java |   21 +-
 .../internal/index/IndexHintJUnitTest.java      |   16 +
 .../query/internal/index/IndexJUnitTest.java    |   21 +-
 .../index/IndexMaintainceJUnitTest.java         |   21 +-
 .../index/IndexMaintenanceJUnitTest.java        |   21 +-
 .../index/IndexStatisticsJUnitTest.java         |   21 +-
 .../IndexTrackingQueryObserverDUnitTest.java    |   21 +-
 .../IndexTrackingQueryObserverJUnitTest.java    |   21 +-
 .../query/internal/index/IndexUseJUnitTest.java |   73 +-
 .../IndexedMergeEquiJoinScenariosJUnitTest.java |   21 +-
 ...itializeIndexEntryDestroyQueryDUnitTest.java |   21 +-
 .../internal/index/MapIndexStoreJUnitTest.java  |   21 +-
 .../MapRangeIndexMaintenanceJUnitTest.java      |   71 +-
 .../index/MultiIndexCreationDUnitTest.java      |   23 +-
 .../NewDeclarativeIndexCreationJUnitTest.java   |   21 +-
 .../index/PdxCopyOnReadQueryJUnitTest.java      |   17 +
 ...gRegionCreationIndexUpdateTypeJUnitTest.java |   21 +-
 .../PutAllWithIndexPerfDUnitDisabledTest.java   |   24 +-
 .../internal/index/RangeIndexAPIJUnitTest.java  |   55 +-
 .../PRBasicIndexCreationDUnitTest.java          |   20 +-
 .../PRBasicIndexCreationDeadlockDUnitTest.java  |   20 +-
 .../PRBasicMultiIndexCreationDUnitTest.java     |   20 +-
 .../partitioned/PRBasicQueryDUnitTest.java      |   20 +-
 .../PRBasicRemoveIndexDUnitTest.java            |   21 +-
 .../PRColocatedEquiJoinDUnitTest.java           |   21 +-
 .../partitioned/PRIndexStatisticsJUnitTest.java |   21 +-
 .../partitioned/PRInvalidQueryDUnitTest.java    |   20 +-
 .../partitioned/PRInvalidQueryJUnitTest.java    |   21 +-
 .../partitioned/PRQueryCacheCloseDUnitTest.java |   20 +-
 .../PRQueryCacheClosedJUnitTest.java            |   21 +-
 .../query/partitioned/PRQueryDUnitHelper.java   |   20 +-
 .../query/partitioned/PRQueryDUnitTest.java     |   28 +-
 .../query/partitioned/PRQueryJUnitTest.java     |   51 +-
 .../partitioned/PRQueryNumThreadsJUnitTest.java |   21 +-
 .../query/partitioned/PRQueryPerfDUnitTest.java |   20 +-
 .../PRQueryRegionCloseDUnitTest.java            |   20 +-
 .../PRQueryRegionClosedJUnitTest.java           |   21 +-
 .../PRQueryRegionDestroyedDUnitTest.java        |   20 +-
 .../PRQueryRegionDestroyedJUnitTest.java        |   21 +-
 .../PRQueryRemoteNodeExceptionDUnitTest.java    |   21 +-
 .../gemfire/cache/query/transaction/Person.java |   21 +-
 .../query/transaction/QueryAndJtaJUnitTest.java |   21 +-
 .../internal/ConnectionCountProbeJUnitTest.java |   21 +-
 .../cache/snapshot/CacheSnapshotJUnitTest.java  |   21 +-
 .../snapshot/ParallelSnapshotDUnitTest.java     |   21 +-
 .../gemfire/cache/snapshot/RegionGenerator.java |   21 +-
 .../cache/snapshot/RegionSnapshotJUnitTest.java |   21 +-
 .../snapshot/SnapshotByteArrayDUnitTest.java    |   21 +-
 .../cache/snapshot/SnapshotDUnitTest.java       |   21 +-
 .../snapshot/SnapshotPerformanceDUnitTest.java  |   21 +-
 .../cache/snapshot/SnapshotTestCase.java        |   21 +-
 .../cache/snapshot/WanSnapshotJUnitTest.java    |   21 +-
 .../cache/util/PasswordUtilJUnitTest.java       |   21 +-
 .../cache30/BridgeMembershipDUnitTest.java      | 1597 ------
 .../BridgeMembershipSelectorDUnitTest.java      |   16 -
 .../gemfire/cache30/BridgeTestCase.java         |  376 --
 .../gemfire/cache30/BridgeWriterDUnitTest.java  |  418 --
 .../cache30/BridgeWriterSelectorDUnitTest.java  |   16 -
 .../gemfire/cache30/Bug34387DUnitTest.java      |   21 +-
 .../gemfire/cache30/Bug34948DUnitTest.java      |   21 +-
 .../gemfire/cache30/Bug35214DUnitTest.java      |   21 +-
 .../gemfire/cache30/Bug38013DUnitTest.java      |   21 +-
 .../gemfire/cache30/Bug38741DUnitTest.java      |   31 +-
 .../cache30/Bug40255JUnitDisabledTest.java      |   22 +-
 .../cache30/Bug40662JUnitDisabledTest.java      |   21 +-
 .../gemfire/cache30/Bug44418JUnitTest.java      |   21 +-
 .../gemfire/cache30/CacheCloseDUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheListenerTestCase.java  |   21 +-
 .../gemfire/cache30/CacheLoaderTestCase.java    |   23 +-
 .../gemfire/cache30/CacheLogRollDUnitTest.java  |   21 +-
 .../gemfire/cache30/CacheMapTxnDUnitTest.java   |   22 +-
 ...cheRegionsReliablityStatsCheckDUnitTest.java |   21 +-
 .../cache30/CacheSerializableRunnable.java      |   21 +-
 .../cache30/CacheStatisticsDUnitTest.java       |   21 +-
 .../gemstone/gemfire/cache30/CacheTestCase.java |  148 +-
 .../gemfire/cache30/CacheWriterTestCase.java    |   21 +-
 .../cache30/CacheXMLPartitionResolver.java      |   21 +-
 .../gemfire/cache30/CacheXml30DUnitTest.java    |  148 +-
 .../gemfire/cache30/CacheXml40DUnitTest.java    |   45 +-
 .../gemfire/cache30/CacheXml41DUnitTest.java    |   49 +-
 .../gemfire/cache30/CacheXml45DUnitTest.java    |   25 +-
 .../gemfire/cache30/CacheXml51DUnitTest.java    |   29 +-
 .../gemfire/cache30/CacheXml55DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml57DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml58DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml60DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml61DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml65DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml66DUnitTest.java    |   21 +-
 .../gemfire/cache30/CacheXml70DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml80DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml81DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXml90DUnitTest.java    |   22 +-
 .../gemfire/cache30/CacheXmlTestCase.java       |   16 +
 .../cache30/CachedAllEventsDUnitTest.java       |   21 +-
 .../gemfire/cache30/CallbackArgDUnitTest.java   |   21 +-
 .../cache30/CertifiableTestCacheListener.java   |   24 +-
 .../cache30/ClearMultiVmCallBkDUnitTest.java    |   22 +-
 .../gemfire/cache30/ClearMultiVmDUnitTest.java  |   22 +-
 .../cache30/ClientMembershipDUnitTest.java      | 1660 +++++++
 .../ClientMembershipSelectorDUnitTest.java      |   32 +
 .../ClientRegisterInterestDUnitTest.java        |  427 ++
 ...ClientRegisterInterestSelectorDUnitTest.java |   32 +
 .../cache30/ClientServerCCEDUnitTest.java       |   23 +-
 .../gemfire/cache30/ClientServerTestCase.java   |  385 ++
 .../ConcurrentLeaveDuringGIIDUnitTest.java      |   21 +-
 ...ibutedNoAckAsyncOverflowRegionDUnitTest.java |   22 +-
 ...iskDistributedNoAckAsyncRegionDUnitTest.java |   22 +-
 .../DiskDistributedNoAckRegionTestCase.java     |   22 +-
 ...ributedNoAckSyncOverflowRegionDUnitTest.java |   22 +-
 .../gemfire/cache30/DiskRegionDUnitTest.java    |   21 +-
 .../gemfire/cache30/DiskRegionTestImpl.java     |   22 +-
 .../cache30/DistAckMapMethodsDUnitTest.java     |   22 +-
 ...ckOverflowRegionCCECompressionDUnitTest.java |   61 -
 ...istributedAckOverflowRegionCCEDUnitTest.java |   21 +-
 ...tedAckOverflowRegionCCEOffHeapDUnitTest.java |   16 +
 ...PersistentRegionCCECompressionDUnitTest.java |   62 -
 ...tributedAckPersistentRegionCCEDUnitTest.java |   40 +-
 ...dAckPersistentRegionCCEOffHeapDUnitTest.java |   16 +
 .../DistributedAckRegionCCEDUnitTest.java       |   30 +-
 ...DistributedAckRegionCCEOffHeapDUnitTest.java |   16 +
 ...istributedAckRegionCompressionDUnitTest.java |   21 +-
 .../cache30/DistributedAckRegionDUnitTest.java  |   23 +-
 .../DistributedAckRegionOffHeapDUnitTest.java   |   16 +
 .../DistributedMulticastRegionDUnitTest.java    |  213 +
 .../DistributedNoAckRegionCCEDUnitTest.java     |   39 +-
 ...stributedNoAckRegionCCEOffHeapDUnitTest.java |   16 +
 ...tributedNoAckRegionCompressionDUnitTest.java |   62 -
 .../DistributedNoAckRegionDUnitTest.java        |   35 +-
 .../DistributedNoAckRegionOffHeapDUnitTest.java |   16 +
 .../gemfire/cache30/DynamicRegionDUnitTest.java |   21 +-
 .../gemfire/cache30/GlobalLockingDUnitTest.java |   21 +-
 .../cache30/GlobalRegionCCEDUnitTest.java       |   25 +-
 .../GlobalRegionCCEOffHeapDUnitTest.java        |   16 +
 .../GlobalRegionCompressionDUnitTest.java       |   61 -
 .../gemfire/cache30/GlobalRegionDUnitTest.java  |   21 +-
 .../cache30/GlobalRegionOffHeapDUnitTest.java   |   16 +
 .../cache30/LRUEvictionControllerDUnitTest.java |   21 +-
 .../gemfire/cache30/LocalRegionDUnitTest.java   |   21 +-
 .../MemLRUEvictionControllerDUnitTest.java      |   21 +-
 .../gemfire/cache30/MultiVMRegionTestCase.java  |  716 ++-
 .../gemfire/cache30/MyGatewayEventFilter1.java  |   21 +-
 .../gemfire/cache30/MyGatewayEventFilter2.java  |   23 +-
 .../cache30/MyGatewayTransportFilter1.java      |   21 +-
 .../cache30/MyGatewayTransportFilter2.java      |   21 +-
 .../OffHeapLRUEvictionControllerDUnitTest.java  |   21 +-
 .../PRBucketSynchronizationDUnitTest.java       |   26 +-
 .../PartitionedRegionCompressionDUnitTest.java  |   25 +-
 .../cache30/PartitionedRegionDUnitTest.java     |   21 +-
 ...tionedRegionMembershipListenerDUnitTest.java |   22 +-
 .../PartitionedRegionOffHeapDUnitTest.java      |   16 +
 .../cache30/PreloadedRegionTestCase.java        |   21 +-
 .../gemfire/cache30/ProxyDUnitTest.java         |   21 +-
 .../cache30/PutAllCallBkRemoteVMDUnitTest.java  |   22 +-
 .../cache30/PutAllCallBkSingleVMDUnitTest.java  |   22 +-
 .../gemfire/cache30/PutAllMultiVmDUnitTest.java |   22 +-
 .../gemfire/cache30/QueueMsgDUnitTest.java      |   21 +-
 .../cache30/RRSynchronizationDUnitTest.java     |   21 +-
 .../gemfire/cache30/ReconnectDUnitTest.java     |  173 +-
 .../ReconnectedCacheServerDUnitTest.java        |   31 +-
 .../cache30/RegionAttributesTestCase.java       |   21 +-
 .../cache30/RegionExpirationDUnitTest.java      |   21 +-
 .../RegionMembershipListenerDUnitTest.java      |   42 +-
 .../RegionReliabilityDistAckDUnitTest.java      |   21 +-
 .../RegionReliabilityDistNoAckDUnitTest.java    |   21 +-
 .../RegionReliabilityGlobalDUnitTest.java       |   21 +-
 .../RegionReliabilityListenerDUnitTest.java     |   21 +-
 .../cache30/RegionReliabilityTestCase.java      |  105 +-
 .../gemfire/cache30/RegionTestCase.java         | 1031 ++--
 .../gemfire/cache30/ReliabilityTestCase.java    |   21 +-
 .../cache30/RemoveAllMultiVmDUnitTest.java      |   22 +-
 .../gemfire/cache30/RequiredRolesDUnitTest.java |   21 +-
 .../cache30/RolePerformanceDUnitTest.java       |   21 +-
 .../gemfire/cache30/SearchAndLoadDUnitTest.java |   21 +-
 .../cache30/SlowRecDUnitDisabledTest.java       |   21 +-
 .../gemfire/cache30/TXDistributedDUnitTest.java |  119 +-
 .../gemfire/cache30/TXOrderDUnitTest.java       |   21 +-
 .../cache30/TXRestrictionsDUnitTest.java        |   21 +-
 .../gemfire/cache30/TestCacheCallback.java      |   34 +-
 .../gemfire/cache30/TestCacheListener.java      |   21 +-
 .../gemfire/cache30/TestCacheLoader.java        |   21 +-
 .../gemfire/cache30/TestCacheWriter.java        |   21 +-
 .../gemfire/cache30/TestDiskRegion.java         |   21 +-
 .../gemstone/gemfire/cache30/TestHeapLRU.java   |   21 +-
 .../gemfire/cache30/TestPdxSerializer.java      |   21 +-
 .../cache30/TestTransactionListener.java        |   21 +-
 .../gemfire/cache30/TestTransactionWriter.java  |   21 +-
 .../AnalyzeSerializablesJUnitTest.java          |   76 +-
 .../codeAnalysis/ClassAndMethodDetails.java     |   23 +-
 .../gemfire/codeAnalysis/ClassAndMethods.java   |   23 +-
 .../codeAnalysis/ClassAndVariableDetails.java   |   23 +-
 .../gemfire/codeAnalysis/ClassAndVariables.java |   23 +-
 .../codeAnalysis/CompiledClassUtils.java        |   23 +-
 .../codeAnalysis/decode/CompiledAttribute.java  |   21 +-
 .../codeAnalysis/decode/CompiledClass.java      |   25 +-
 .../codeAnalysis/decode/CompiledCode.java       |   21 +-
 .../codeAnalysis/decode/CompiledField.java      |   21 +-
 .../codeAnalysis/decode/CompiledMethod.java     |   21 +-
 .../gemfire/codeAnalysis/decode/cp/Cp.java      |   21 +-
 .../gemfire/codeAnalysis/decode/cp/CpClass.java |   21 +-
 .../codeAnalysis/decode/cp/CpDouble.java        |   21 +-
 .../codeAnalysis/decode/cp/CpFieldref.java      |   18 +-
 .../gemfire/codeAnalysis/decode/cp/CpFloat.java |   18 +-
 .../codeAnalysis/decode/cp/CpInteger.java       |   18 +-
 .../decode/cp/CpInterfaceMethodref.java         |   18 +-
 .../gemfire/codeAnalysis/decode/cp/CpLong.java  |   21 +-
 .../codeAnalysis/decode/cp/CpMethodref.java     |   18 +-
 .../codeAnalysis/decode/cp/CpNameAndType.java   |   18 +-
 .../codeAnalysis/decode/cp/CpString.java        |   18 +-
 .../gemfire/codeAnalysis/decode/cp/CpUtf8.java  |   21 +-
 .../AbstractLauncherIntegrationJUnitTest.java   |   71 +
 .../distributed/AbstractLauncherJUnitTest.java  |   59 +-
 .../AbstractLauncherJUnitTestCase.java          |   16 +
 .../AbstractLauncherServiceStatusJUnitTest.java |   21 +-
 .../AbstractLocatorLauncherJUnitTestCase.java   |   16 +
 .../AbstractServerLauncherJUnitTestCase.java    |   20 +-
 .../gemfire/distributed/AuthInitializer.java    |   23 +-
 .../distributed/CommonLauncherTestSuite.java    |   57 -
 .../distributed/DistributedMemberDUnitTest.java |   21 +-
 .../DistributedSystemConnectPerf.java           |   21 +-
 .../distributed/DistributedSystemDUnitTest.java |  145 +-
 .../DistributedSystemIntegrationJUnitTest.java  |   91 +
 .../distributed/DistributedSystemJUnitTest.java |   78 +
 .../distributed/DistributedTestSuite.java       |   16 +
 .../distributed/HostedLocatorsDUnitTest.java    |   16 +
 .../gemfire/distributed/JGroupsJUnitTest.java   |  171 -
 .../LauncherMemberMXBeanJUnitTest.java          |   16 +
 .../gemfire/distributed/LauncherTestSuite.java  |   16 +
 .../gemfire/distributed/LocatorDUnitTest.java   |  571 +--
 .../gemfire/distributed/LocatorJUnitTest.java   |  117 +-
 .../LocatorLauncherIntegrationJUnitTest.java    |  248 +
 .../distributed/LocatorLauncherJUnitTest.java   |  179 +-
 .../LocatorLauncherLocalFileJUnitTest.java      |   16 +
 .../LocatorLauncherLocalJUnitTest.java          |  108 +-
 .../LocatorLauncherRemoteFileJUnitTest.java     |   28 +-
 .../LocatorLauncherRemoteJUnitTest.java         |   77 +-
 .../gemfire/distributed/MyAuthenticator.java    |   23 +-
 .../gemfire/distributed/MyPrincipal.java        |   18 +-
 .../gemfire/distributed/RoleDUnitTest.java      |   21 +-
 .../ServerLauncherIntegrationJUnitTest.java     |  312 ++
 .../distributed/ServerLauncherJUnitTest.java    |  235 +-
 .../ServerLauncherLocalFileJUnitTest.java       |   20 +-
 .../ServerLauncherLocalJUnitTest.java           |  187 +-
 .../ServerLauncherRemoteFileJUnitTest.java      |   18 +-
 .../ServerLauncherRemoteJUnitTest.java          |   42 +-
 .../ServerLauncherWithSpringJUnitTest.java      |   21 +
 .../distributed/SystemAdminDUnitTest.java       |   21 +-
 .../AtomicLongWithTerminalStateJUnitTest.java   |   21 +-
 .../distributed/internal/Bug40751DUnitTest.java |   21 +-
 .../ConsoleDistributionManagerDUnitTest.java    |   23 +-
 .../distributed/internal/DateMessage.java       |   21 +-
 .../internal/DistributionAdvisorDUnitTest.java  |   21 +-
 .../internal/DistributionManagerDUnitTest.java  |  166 +-
 ...istributionManagerTimeDUnitDisabledTest.java |  211 -
 .../GemFireTimeSyncServiceDUnitTest.java        |  260 -
 .../InternalDistributedSystemJUnitTest.java     |   52 +-
 .../gemfire/distributed/internal/LDM.java       |   21 +-
 .../internal/LocalDistributionManagerTest.java  |  475 --
 .../internal/LocatorLoadSnapshotJUnitTest.java  |   21 +-
 .../internal/ProduceDateMessages.java           |   21 +-
 .../internal/ProductUseLogDUnitTest.java        |   28 +-
 .../internal/ProductUseLogJUnitTest.java        |   21 +-
 .../internal/ServerLocatorJUnitTest.java        |   20 +-
 .../internal/SharedConfigurationJUnitTest.java  |   21 +-
 .../internal/StartupMessageDataJUnitTest.java   |   21 +-
 .../deadlock/DeadlockDetectorJUnitTest.java     |   21 +-
 .../deadlock/DependencyGraphJUnitTest.java      |   21 +-
 .../GemFireDeadlockDetectorDUnitTest.java       |   68 +-
 .../deadlock/UnsafeThreadLocalJUnitTest.java    |   21 +-
 .../locks/CollaborationJUnitDisabledTest.java   |   21 +-
 .../internal/locks/DLockGrantorHelper.java      |   21 +-
 .../locks/DLockReentrantLockJUnitTest.java      |   84 +
 ...entrantReadWriteWriteShareLockJUnitTest.java |   21 +-
 .../membership/MembershipJUnitTest.java         |  423 +-
 .../internal/membership/NetViewJUnitTest.java   |  274 ++
 .../membership/gms/GMSMemberJUnitTest.java      |  164 +
 .../membership/gms/MembershipManagerHelper.java |  172 +
 .../gms/auth/GMSAuthenticatorJUnitTest.java     |  316 ++
 .../gms/fd/GMSHealthMonitorJUnitTest.java       |  644 +++
 .../locator/GMSLocatorRecoveryJUnitTest.java    |  186 +
 .../gms/membership/GMSJoinLeaveJUnitTest.java   | 1095 +++++
 .../gms/membership/StatRecorderJUnitTest.java   |  227 +
 .../messenger/GMSQuorumCheckerJUnitTest.java    |  361 ++
 .../membership/gms/messenger/InterceptUDP.java  |  109 +
 .../messenger/JGroupsMessengerJUnitTest.java    |  836 ++++
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |  437 ++
 .../jgroup/MembershipManagerHelper.java         |  180 -
 .../StreamingOperationManyDUnitTest.java        |   29 +-
 .../StreamingOperationOneDUnitTest.java         |   29 +-
 .../tcpserver/LocatorVersioningJUnitTest.java   |  205 -
 ...cpServerBackwardCompatDUnitDisabledTest.java |   80 +-
 .../tcpserver/TcpServerJUnitDisabledTest.java   |  203 -
 .../internal/tcpserver/TcpServerJUnitTest.java  |  221 +
 .../support/DistributedSystemAdapter.java       |   21 +-
 .../gemfire/disttx/CacheMapDistTXDUnitTest.java |   25 +-
 .../gemfire/disttx/DistTXDebugDUnitTest.java    |   31 +-
 .../disttx/DistTXDistributedTestSuite.java      |   21 +-
 .../gemfire/disttx/DistTXExpiryJUnitTest.java   |   16 +
 .../gemfire/disttx/DistTXJUnitTest.java         |   16 +
 .../disttx/DistTXManagerImplJUnitTest.java      |   16 +
 .../gemfire/disttx/DistTXOrderDUnitTest.java    |   29 +-
 .../disttx/DistTXPersistentDebugDUnitTest.java  |   23 +-
 .../DistTXReleasesOffHeapOnCloseJUnitTest.java  |   16 +
 .../disttx/DistTXRestrictionsDUnitTest.java     |   17 +-
 .../disttx/DistTXWithDeltaDUnitTest.java        |   38 +
 .../gemfire/disttx/DistTXWriterJUnitTest.java   |   16 +
 .../disttx/DistTXWriterOOMEJUnitTest.java       |   16 +
 .../disttx/DistributedTransactionDUnitTest.java |  832 +++-
 .../gemfire/disttx/PRDistTXDUnitTest.java       |   65 +
 .../gemfire/disttx/PRDistTXJUnitTest.java       |   16 +
 .../disttx/PRDistTXWithVersionsDUnitTest.java   |   66 +
 ...entPartitionedRegionWithDistTXDUnitTest.java |   16 +
 .../internal/AbstractConfigJUnitTest.java       |  114 +
 .../gemfire/internal/ArrayEqualsJUnitTest.java  |   21 +-
 .../gemfire/internal/AvailablePortHelper.java   |   25 +-
 .../internal/AvailablePortJUnitTest.java        |   23 +-
 ...wardCompatibilitySerializationDUnitTest.java |  300 ++
 ...wardCompatibilitySerializationJUnitTest.java |  306 --
 .../gemfire/internal/Bug49856JUnitTest.java     |   21 +-
 .../gemfire/internal/Bug51616JUnitTest.java     |   16 +
 .../gemfire/internal/ByteArrayData.java         |   21 +-
 .../gemstone/gemfire/internal/ClassBuilder.java |   21 +-
 .../ClassNotFoundExceptionDUnitTest.java        |   21 +-
 .../internal/ClassPathLoaderJUnitTest.java      |   21 +-
 .../internal/CopyOnWriteHashSetJUnitTest.java   |   21 +-
 .../internal/DataSerializableJUnitTest.java     |   56 +-
 .../gemstone/gemfire/internal/FDDUnitTest.java  |  265 -
 .../gemfire/internal/FileUtilJUnitTest.java     |   21 +-
 .../internal/GemFireStatSamplerJUnitTest.java   |   21 +-
 .../GemFireVersionIntegrationJUnitTest.java     |   21 +-
 .../internal/GemFireVersionJUnitTest.java       |   21 +-
 .../internal/HeapDataOutputStreamJUnitTest.java |   21 +-
 .../gemfire/internal/InlineKeyJUnitTest.java    |   21 +-
 .../gemfire/internal/JSSESocketJUnitTest.java   |   23 +-
 .../internal/JarClassLoaderJUnitTest.java       |   22 +-
 .../gemfire/internal/JarDeployerDUnitTest.java  |   22 +-
 .../com/gemstone/gemfire/internal/JavaExec.java |   21 +-
 .../gemfire/internal/LineWrapUnitJUnitTest.java |   21 +-
 .../gemstone/gemfire/internal/LongBuffer.java   |   21 +-
 .../gemfire/internal/NanoTimer2JUnitTest.java   |   79 -
 .../gemfire/internal/NanoTimerJUnitTest.java    |  142 +-
 .../gemfire/internal/ObjIdMapJUnitTest.java     |   21 +-
 .../internal/OneTaskOnlyDecoratorJUnitTest.java |   21 +-
 .../internal/PdxDeleteFieldDUnitTest.java       |   29 +-
 .../internal/PdxDeleteFieldJUnitTest.java       |   16 +
 .../gemfire/internal/PdxRenameDUnitTest.java    |   31 +-
 .../gemfire/internal/PdxRenameJUnitTest.java    |   16 +
 .../PutAllOperationContextJUnitTest.java        |   21 +-
 .../internal/SSLConfigIntegrationJUnitTest.java |   16 +
 .../gemfire/internal/SSLConfigJUnitTest.java    |   20 +-
 ...hreadPoolExecutorWithKeepAliveJUnitTest.java |   21 +-
 .../internal/SimpleStatSamplerJUnitTest.java    |   21 +-
 .../gemfire/internal/SocketCloserJUnitTest.java |  196 +
 .../internal/SocketCloserWithWaitJUnitTest.java |   38 +
 .../StatArchiveWriterReaderJUnitTest.java       |   21 +-
 .../gemfire/internal/StatSamplerJUnitTest.java  |   21 +-
 .../gemfire/internal/StatSamplerTestCase.java   |   21 +-
 .../internal/UniqueIdGeneratorJUnitTest.java    |   21 +-
 .../internal/cache/AbstractRegionJUnitTest.java |   21 +-
 .../gemfire/internal/cache/BackupDUnitTest.java |   21 +-
 .../gemfire/internal/cache/BackupJUnitTest.java |   21 +-
 .../internal/cache/Bug33359DUnitTest.java       |   21 +-
 .../internal/cache/Bug33726DUnitTest.java       |   21 +-
 .../internal/cache/Bug33726JUnitTest.java       |   23 +-
 .../internal/cache/Bug34011JUnitTest.java       |  199 -
 .../Bug34179TooManyFilesOpenJUnitTest.java      |   21 +-
 .../internal/cache/Bug34583JUnitTest.java       |   21 +-
 .../internal/cache/Bug37241DUnitTest.java       |   21 +-
 .../internal/cache/Bug37244JUnitTest.java       |   21 +-
 .../internal/cache/Bug37377DUnitTest.java       |   21 +-
 .../internal/cache/Bug37500JUnitTest.java       |   21 +-
 .../internal/cache/Bug39079DUnitTest.java       |   27 +-
 .../internal/cache/Bug40299DUnitTest.java       |   21 +-
 .../internal/cache/Bug40632DUnitTest.java       |   21 +-
 .../internal/cache/Bug41091DUnitTest.java       |   21 +-
 .../internal/cache/Bug41733DUnitTest.java       |   21 +-
 .../internal/cache/Bug41957DUnitTest.java       |   35 +-
 .../internal/cache/Bug42010StatsDUnitTest.java  |   21 +-
 .../internal/cache/Bug42055DUnitTest.java       |   21 +-
 .../internal/cache/Bug45164DUnitTest.java       |   21 +-
 .../internal/cache/Bug45934DUnitTest.java       |   21 +-
 .../internal/cache/Bug47667DUnitTest.java       |   21 +-
 .../internal/cache/Bug48182JUnitTest.java       |   16 +
 .../internal/cache/CacheAdvisorDUnitTest.java   |   21 +-
 .../cache/CacheLifecycleListenerJUnitTest.java  |   21 +-
 .../internal/cache/CacheServiceJUnitTest.java   |   59 +
 .../cache/ChunkValueWrapperJUnitTest.java       |   20 +-
 .../internal/cache/ClearDAckDUnitTest.java      |   21 +-
 .../internal/cache/ClearGlobalDUnitTest.java    |   21 +-
 ...ssagesRegionCreationAndDestroyJUnitTest.java |   35 +-
 .../cache/ClientServerGetAllDUnitTest.java      |  100 +-
 ...ServerInvalidAndDestroyedEntryDUnitTest.java |   21 +-
 .../ClientServerTransactionCCEDUnitTest.java    |   21 +-
 .../cache/ClientServerTransactionDUnitTest.java |  271 +-
 .../cache/ComplexDiskRegionJUnitTest.java       |   21 +-
 .../ConcurrentDestroySubRegionDUnitTest.java    |   21 +-
 ...entFlushingAndRegionOperationsJUnitTest.java |   21 +-
 .../cache/ConcurrentMapLocalJUnitTest.java      |   21 +-
 .../cache/ConcurrentMapOpsDUnitTest.java        |   98 +-
 .../ConcurrentRegionOperationsJUnitTest.java    |   28 +-
 ...rentRollingAndRegionOperationsJUnitTest.java |   21 +-
 .../internal/cache/ConflationJUnitTest.java     |   21 +-
 .../cache/ConnectDisconnectDUnitTest.java       |   57 +-
 .../cache/CustomerIDPartitionResolver.java      |   21 +-
 .../internal/cache/DeltaFaultInDUnitTest.java   |   21 +-
 .../cache/DeltaPropagationDUnitTest.java        |   29 +-
 .../cache/DeltaPropagationStatsDUnitTest.java   |   23 +-
 .../internal/cache/DeltaSizingDUnitTest.java    |   21 +-
 .../gemfire/internal/cache/DiskIFJUnitTest.java |   21 +-
 .../gemfire/internal/cache/DiskIdJUnitTest.java |   21 +-
 .../internal/cache/DiskInitFileJUnitTest.java   |   21 +-
 .../cache/DiskOfflineCompactionJUnitTest.java   |   21 +-
 .../internal/cache/DiskOldAPIsJUnitTest.java    |   21 +-
 ...iskRandomOperationsAndRecoveryJUnitTest.java |   21 +-
 .../cache/DiskRegByteArrayDUnitTest.java        |   21 +-
 .../cache/DiskRegCacheXmlJUnitTest.java         |   21 +-
 .../DiskRegCachexmlGeneratorJUnitTest.java      |   21 +-
 .../internal/cache/DiskRegCbkChkJUnitTest.java  |   21 +-
 .../DiskRegOplogSwtchingAndRollerJUnitTest.java |   21 +-
 .../cache/DiskRegRecoveryJUnitTest.java         |   21 +-
 .../cache/DiskRegionAsyncRecoveryJUnitTest.java |   21 +-
 ...RegionChangingRegionAttributesJUnitTest.java |   21 +-
 .../cache/DiskRegionClearJUnitTest.java         |   21 +-
 .../internal/cache/DiskRegionHelperFactory.java |   21 +-
 .../DiskRegionIllegalArguementsJUnitTest.java   |   21 +-
 ...iskRegionIllegalCacheXMLvaluesJUnitTest.java |   21 +-
 .../internal/cache/DiskRegionJUnitTest.java     |   60 +-
 .../internal/cache/DiskRegionProperties.java    |   21 +-
 .../internal/cache/DiskRegionTestingBase.java   |   21 +-
 .../cache/DiskStoreFactoryJUnitTest.java        |   21 +-
 .../cache/DiskWriteAttributesJUnitTest.java     |   21 +-
 ...DistrbutedRegionProfileOffHeapDUnitTest.java |   16 +
 .../cache/DistributedCacheTestCase.java         |   21 +-
 .../cache/EnumListenerEventJUnitTest.java       |   21 +-
 .../internal/cache/EventTrackerDUnitTest.java   |   29 +-
 .../cache/EvictionDUnitDisabledTest.java        |   21 +-
 .../cache/EvictionObjectSizerDUnitTest.java     |   21 +-
 .../internal/cache/EvictionStatsDUnitTest.java  |   21 +-
 .../internal/cache/EvictionTestBase.java        |   21 +-
 .../internal/cache/FaultingInJUnitTest.java     |   21 +-
 .../cache/FixedPRSinglehopDUnitTest.java        |   21 +-
 .../internal/cache/GIIDeltaDUnitTest.java       |   23 +-
 .../internal/cache/GIIFlowControlDUnitTest.java |   21 +-
 .../internal/cache/GridAdvisorDUnitTest.java    |  127 +-
 .../internal/cache/HABug36773DUnitTest.java     |   29 +-
 .../HAOverflowMemObjectSizerDUnitTest.java      |   33 +-
 .../HDFSQueueRegionOperationsJUnitTest.java     |   33 -
 ...FSQueueRegionOperationsOffHeapJUnitTest.java |   54 -
 .../cache/HDFSRegionOperationsJUnitTest.java    |  545 ---
 .../HDFSRegionOperationsOffHeapJUnitTest.java   |   78 -
 .../cache/IncrementalBackupDUnitTest.java       |   21 +-
 .../cache/InterruptClientServerDUnitTest.java   |   21 +-
 .../internal/cache/InterruptDiskJUnitTest.java  |   21 +-
 ...InterruptsConserveSocketsFalseDUnitTest.java |   16 +
 .../internal/cache/InterruptsDUnitTest.java     |   21 +-
 .../internal/cache/IteratorDUnitTest.java       |   21 +-
 .../LIFOEvictionAlgoEnabledRegionJUnitTest.java |   21 +-
 ...victionAlgoMemoryEnabledRegionJUnitTest.java |  140 +-
 .../internal/cache/MapClearGIIDUnitTest.java    |   21 +-
 .../internal/cache/MapInterface2JUnitTest.java  |   21 +-
 .../internal/cache/MapInterfaceJUnitTest.java   |   21 +-
 .../internal/cache/MockCacheService.java        |   24 +
 .../internal/cache/MockCacheServiceImpl.java    |   39 +
 .../MultipleOplogsRollingFeatureJUnitTest.java  |   21 +-
 .../cache/NetSearchMessagingDUnitTest.java      |   51 +-
 .../cache/OffHeapEvictionDUnitTest.java         |   23 +-
 .../cache/OffHeapEvictionStatsDUnitTest.java    |   21 +-
 .../gemfire/internal/cache/OffHeapTestUtil.java |   26 +-
 .../cache/OfflineSnapshotJUnitTest.java         |   21 +-
 .../gemfire/internal/cache/OldVLJUnitTest.java  |   21 +-
 .../cache/OldValueImporterTestBase.java         |  181 +
 .../cache/OplogEntryIdMapJUnitTest.java         |   21 +-
 .../cache/OplogEntryIdSetJUnitTest.java         |   21 +-
 .../gemfire/internal/cache/OplogJUnitTest.java  |   67 +-
 .../internal/cache/OplogRVVJUnitTest.java       |   21 +-
 .../cache/OrderedTombstoneMapJUnitTest.java     |   21 +-
 .../cache/P2PDeltaPropagationDUnitTest.java     |   21 +-
 .../internal/cache/PRBadToDataDUnitTest.java    |   21 +-
 .../cache/PRConcurrentMapOpsJUnitTest.java      |   21 +-
 .../cache/PRDataStoreMemoryJUnitTest.java       |   21 +-
 .../PRDataStoreMemoryOffHeapJUnitTest.java      |   16 +
 .../gemfire/internal/cache/PRTXJUnitTest.java   |   21 +-
 .../cache/PartitionAttributesImplJUnitTest.java |   16 +
 .../cache/PartitionListenerDUnitTest.java       |   21 +-
 ...dRegionAPIConserveSocketsFalseDUnitTest.java |   21 +-
 .../cache/PartitionedRegionAPIDUnitTest.java    |   20 +-
 .../PartitionedRegionAsSubRegionDUnitTest.java  |   20 +-
 ...gionBucketCreationDistributionDUnitTest.java |   20 +-
 .../PartitionedRegionCacheCloseDUnitTest.java   |   20 +-
 ...rtitionedRegionCacheLoaderForRootRegion.java |   21 +-
 ...artitionedRegionCacheLoaderForSubRegion.java |   21 +-
 ...rtitionedRegionCacheXMLExampleDUnitTest.java |   21 +-
 .../PartitionedRegionCreationDUnitTest.java     |   20 +-
 .../PartitionedRegionCreationJUnitTest.java     |   20 +-
 .../cache/PartitionedRegionDUnitTestCase.java   |   20 +-
 .../PartitionedRegionDataStoreJUnitTest.java    |   20 +-
 ...rtitionedRegionDelayedRecoveryDUnitTest.java |   21 +-
 .../PartitionedRegionDestroyDUnitTest.java      |   20 +-
 .../PartitionedRegionEntryCountDUnitTest.java   |   21 +-
 .../PartitionedRegionEvictionDUnitTest.java     |   21 +-
 .../cache/PartitionedRegionHADUnitTest.java     |   20 +-
 ...onedRegionHAFailureAndRecoveryDUnitTest.java |   20 +-
 .../cache/PartitionedRegionHelperJUnitTest.java |   21 +-
 .../PartitionedRegionInvalidateDUnitTest.java   |   21 +-
 ...artitionedRegionLocalMaxMemoryDUnitTest.java |   20 +-
 ...nedRegionLocalMaxMemoryOffHeapDUnitTest.java |   16 +
 .../PartitionedRegionMultipleDUnitTest.java     |   20 +-
 ...rtitionedRegionOffHeapEvictionDUnitTest.java |   22 +-
 .../cache/PartitionedRegionPRIDDUnitTest.java   |   21 +-
 .../cache/PartitionedRegionQueryDUnitTest.java  |   21 +-
 ...artitionedRegionQueryEvaluatorJUnitTest.java |   21 +-
 ...artitionedRegionRedundancyZoneDUnitTest.java |   25 +-
 ...tionedRegionSerializableObjectJUnitTest.java |   21 +-
 .../PartitionedRegionSingleHopDUnitTest.java    |   84 +-
 ...RegionSingleHopWithServerGroupDUnitTest.java |   21 +-
 ...onedRegionSingleNodeOperationsJUnitTest.java |   20 +-
 .../cache/PartitionedRegionSizeDUnitTest.java   |   20 +-
 .../cache/PartitionedRegionStatsDUnitTest.java  |   20 +-
 .../cache/PartitionedRegionStatsJUnitTest.java  |  341 +-
 .../cache/PartitionedRegionTestHelper.java      |   23 +-
 .../PartitionedRegionTestUtilsDUnitTest.java    |   20 +-
 .../PartitionedRegionWithSameNameDUnitTest.java |   21 +-
 .../PersistentPartitionedRegionJUnitTest.java   |   16 +
 .../internal/cache/PutAllDAckDUnitTest.java     |   21 +-
 .../internal/cache/PutAllGlobalDUnitTest.java   |   21 +-
 .../cache/RegionEntryFlagsJUnitTest.java        |   21 +-
 .../internal/cache/RegionListenerJUnitTest.java |   63 +
 .../cache/RemotePutReplyMessageJUnitTest.java   |   53 +
 .../cache/RemoteTransactionCCEDUnitTest.java    |   16 +
 .../cache/RemoteTransactionDUnitTest.java       |  204 +-
 .../internal/cache/RemoveAllDAckDUnitTest.java  |   21 +-
 .../internal/cache/RemoveDAckDUnitTest.java     |   21 +-
 .../internal/cache/RemoveGlobalDUnitTest.java   |   21 +-
 .../internal/cache/RunCacheInOldGemfire.java    |   21 +-
 .../cache/SimpleDiskRegionJUnitTest.java        |   21 +-
 .../internal/cache/SizingFlagDUnitTest.java     |   21 +-
 .../internal/cache/SnapshotTestUtil.java        |   16 +
 .../internal/cache/SystemFailureDUnitTest.java  |   21 +-
 .../internal/cache/TXManagerImplJUnitTest.java  |   21 +-
 .../cache/TXReservationMgrJUnitTest.java        |   21 +-
 .../gemfire/internal/cache/TestDelta.java       |   21 +-
 .../internal/cache/TestHelperForHydraTests.java |   16 +
 .../internal/cache/TestNonSizerObject.java      |   21 +-
 .../internal/cache/TestObjectSizerImpl.java     |   21 +-
 .../gemfire/internal/cache/TestUtils.java       |   21 +-
 .../cache/TombstoneCreationJUnitTest.java       |   21 +-
 .../cache/TransactionsWithDeltaDUnitTest.java   |   21 +-
 .../internal/cache/UnitTestValueHolder.java     |   18 +-
 .../gemfire/internal/cache/UnzipUtil.java       |   21 +-
 .../internal/cache/UpdateVersionJUnitTest.java  |   21 +-
 .../gemfire/internal/cache/VLJUnitTest.java     |   21 +-
 .../cache/control/FilterByPathJUnitTest.java    |   21 +-
 .../cache/control/MemoryMonitorJUnitTest.java   |   21 +-
 .../control/MemoryMonitorOffHeapJUnitTest.java  |   23 +-
 .../control/MemoryThresholdsJUnitTest.java      |   16 +
 .../control/RebalanceOperationDUnitTest.java    |  639 ++-
 .../control/TestMemoryThresholdListener.java    |   34 +-
 ...skRegOverflowAsyncGetInMemPerfJUnitTest.java |   21 +-
 ...iskRegOverflowAsyncJUnitPerformanceTest.java |   21 +-
 ...lowSyncGetInMemPerfJUnitPerformanceTest.java |   21 +-
 ...DiskRegOverflowSyncJUnitPerformanceTest.java |   21 +-
 ...egionOverflowAsyncRollingOpLogJUnitTest.java |   21 +-
 ...RegionOverflowSyncRollingOpLogJUnitTest.java |   21 +-
 .../DiskRegionPerfJUnitPerformanceTest.java     |   21 +-
 .../DiskRegionPersistOnlySyncJUnitTest.java     |   21 +-
 ...DiskRegionRollOpLogJUnitPerformanceTest.java |   21 +-
 ...ltiThreadedOplogPerJUnitPerformanceTest.java |   21 +-
 .../cache/execute/Bug51193DUnitTest.java        |   28 +-
 .../ClientServerFunctionExecutionDUnitTest.java |   21 +-
 .../execute/ColocationFailoverDUnitTest.java    |   21 +-
 .../cache/execute/CustomResultCollector.java    |   21 +-
 .../execute/CustomerIDPartitionResolver.java    |   21 +-
 ...ributedRegionFunctionExecutionDUnitTest.java |   21 +-
 .../FunctionExecution_ExceptionDUnitTest.java   |   21 +-
 .../execute/FunctionServiceStatsDUnitTest.java  |   64 +-
 .../cache/execute/LocalDataSetDUnitTest.java    |   21 +-
 .../cache/execute/LocalDataSetFunction.java     |   21 +-
 .../execute/LocalDataSetIndexingDUnitTest.java  |   21 +-
 .../LocalFunctionExecutionDUnitTest.java        |   21 +-
 .../MemberFunctionExecutionDUnitTest.java       |   70 +-
 .../MultiRegionFunctionExecutionDUnitTest.java  |   21 +-
 .../execute/MyFunctionExecutionException.java   |   21 +-
 .../cache/execute/MyTransactionFunction.java    |   21 +-
 .../OnGroupsFunctionExecutionDUnitTest.java     |   51 +-
 ...ntServerFunctionExecutionNoAckDUnitTest.java |   21 +-
 ...tServerRegionFunctionExecutionDUnitTest.java |   21 +-
 ...egionFunctionExecutionFailoverDUnitTest.java |   21 +-
 ...onFunctionExecutionNoSingleHopDUnitTest.java |   21 +-
 ...onExecutionSelectorNoSingleHopDUnitTest.java |   21 +-
 ...gionFunctionExecutionSingleHopDUnitTest.java |   23 +-
 .../cache/execute/PRClientServerTestBase.java   |   21 +-
 .../cache/execute/PRColocationDUnitTest.java    |   21 +-
 .../execute/PRCustomPartitioningDUnitTest.java  |   21 +-
 .../execute/PRFunctionExecutionDUnitTest.java   |   21 +-
 .../PRFunctionExecutionTimeOutDUnitTest.java    |   22 +-
 ...ctionExecutionWithResultSenderDUnitTest.java |   21 +-
 .../execute/PRPerformanceTestDUnitTest.java     |   21 +-
 .../cache/execute/PRTransactionDUnitTest.java   |   21 +-
 .../PRTransactionWithVersionsDUnitTest.java     |   16 +
 .../internal/cache/execute/PerfFunction.java    |   21 +-
 .../internal/cache/execute/PerfTxFunction.java  |   21 +-
 .../cache/execute/PerformanceTestFunction.java  |   21 +-
 .../execute/SingleHopGetAllPutAllDUnitTest.java |   39 +-
 .../internal/cache/execute/TestFunction.java    |   21 +-
 .../internal/cache/execute/data/CustId.java     |   21 +-
 .../internal/cache/execute/data/Customer.java   |   21 +-
 .../internal/cache/execute/data/Order.java      |   21 +-
 .../internal/cache/execute/data/OrderId.java    |   21 +-
 .../internal/cache/execute/data/Shipment.java   |   21 +-
 .../internal/cache/execute/data/ShipmentId.java |   21 +-
 .../SimpleExtensionPointJUnitTest.java          |   21 +-
 .../extension/mock/AbstractMockExtension.java   |   21 +-
 .../mock/AbstractMockExtensionXmlGenerator.java |   21 +-
 .../mock/AlterMockCacheExtensionFunction.java   |   21 +-
 .../mock/AlterMockRegionExtensionFunction.java  |   21 +-
 .../mock/CreateMockCacheExtensionFunction.java  |   21 +-
 .../mock/CreateMockRegionExtensionFunction.java |   21 +-
 .../mock/DestroyMockCacheExtensionFunction.java |   23 +-
 .../DestroyMockRegionExtensionFunction.java     |   21 +-
 .../extension/mock/MockCacheExtension.java      |   21 +-
 .../mock/MockCacheExtensionXmlGenerator.java    |   21 +-
 .../extension/mock/MockExtensionCommands.java   |   21 +-
 .../extension/mock/MockExtensionXmlParser.java  |   21 +-
 .../extension/mock/MockRegionExtension.java     |   21 +-
 .../mock/MockRegionExtensionXmlGenerator.java   |   21 +-
 ...gionFunctionFunctionInvocationException.java |   21 +-
 .../functions/DistributedRegionFunction.java    |   21 +-
 .../cache/functions/LocalDataSetFunction.java   |   21 +-
 .../internal/cache/functions/TestFunction.java  |   31 +-
 .../ha/BlockingHARQAddOperationJUnitTest.java   |   21 +-
 .../cache/ha/BlockingHARQStatsJUnitTest.java    |   21 +-
 .../cache/ha/BlockingHARegionJUnitTest.java     |   21 +-
 .../ha/BlockingHARegionQueueJUnitTest.java      |   21 +-
 .../cache/ha/Bug36853EventsExpiryDUnitTest.java |   30 +-
 .../internal/cache/ha/Bug48571DUnitTest.java    |   27 +-
 .../internal/cache/ha/Bug48879DUnitTest.java    |   23 +-
 .../internal/cache/ha/ConflatableObject.java    |   21 +-
 .../cache/ha/EventIdOptimizationDUnitTest.java  |   33 +-
 .../cache/ha/EventIdOptimizationJUnitTest.java  |   20 +-
 .../internal/cache/ha/FailoverDUnitTest.java    |   43 +-
 .../internal/cache/ha/HABugInPutDUnitTest.java  |   29 +-
 .../internal/cache/ha/HAClearDUnitTest.java     |   31 +-
 .../cache/ha/HAConflationDUnitTest.java         |   29 +-
 .../internal/cache/ha/HADuplicateDUnitTest.java |   31 +-
 .../cache/ha/HAEventIdPropagationDUnitTest.java |   31 +-
 .../internal/cache/ha/HAExpiryDUnitTest.java    |   21 +-
 .../internal/cache/ha/HAGIIBugDUnitTest.java    |   21 +-
 .../internal/cache/ha/HAGIIDUnitTest.java       |   39 +-
 .../gemfire/internal/cache/ha/HAHelper.java     |   21 +-
 .../cache/ha/HARQAddOperationJUnitTest.java     |   21 +-
 .../cache/ha/HARQueueNewImplDUnitTest.java      |   87 +-
 .../internal/cache/ha/HARegionDUnitTest.java    |   21 +-
 .../internal/cache/ha/HARegionJUnitTest.java    |   20 +-
 .../cache/ha/HARegionQueueDUnitTest.java        |   21 +-
 .../cache/ha/HARegionQueueJUnitTest.java        |   20 +-
 ...HARegionQueueStartStopJUnitDisabledTest.java |   21 +-
 .../ha/HARegionQueueStartStopJUnitTest.java     |   21 +-
 .../cache/ha/HARegionQueueStatsJUnitTest.java   |   20 +-
 .../cache/ha/HASlowReceiverDUnitTest.java       |   31 +-
 .../ha/OperationsPropagationDUnitTest.java      |   29 +-
 .../internal/cache/ha/PutAllDUnitTest.java      |   33 +-
 .../cache/ha/StatsBugDUnitDisabledTest.java     |   35 +-
 .../cache/ha/TestBlockingHARegionQueue.java     |   21 +-
 .../cache/ha/ThreadIdentifierJUnitTest.java     |   21 +-
 .../cache/locks/TXLockServiceDUnitTest.java     |   23 +-
 .../internal/cache/lru/LRUClockJUnitTest.java   |   21 +-
 .../cache/partitioned/Bug39356DUnitTest.java    |   21 +-
 .../cache/partitioned/Bug43684DUnitTest.java    |   57 +-
 .../cache/partitioned/Bug47388DUnitTest.java    |   21 +-
 .../cache/partitioned/Bug51400DUnitTest.java    |   27 +-
 .../partitioned/ElidedPutAllDUnitTest.java      |   21 +-
 .../OfflineMembersDetailsJUnitTest.java         |   21 +-
 .../partitioned/PartitionResolverDUnitTest.java |   21 +-
 .../PartitionedRegionLoadModelJUnitTest.java    |  126 +-
 .../PartitionedRegionLoaderWriterDUnitTest.java |   21 +-
 ...rtitionedRegionMetaDataCleanupDUnitTest.java |   21 +-
 .../partitioned/PersistPRKRFDUnitTest.java      |   21 +-
 ...tentColocatedPartitionedRegionDUnitTest.java |   23 +-
 .../PersistentPartitionedRegionDUnitTest.java   |   27 +-
 ...tentPartitionedRegionOldConfigDUnitTest.java |   21 +-
 .../PersistentPartitionedRegionTestBase.java    |   21 +-
 ...rtitionedRegionWithTransactionDUnitTest.java |   21 +-
 .../PutPutReplyMessageJUnitTest.java            |   54 +
 .../cache/partitioned/ShutdownAllDUnitTest.java |   33 +-
 ...treamingPartitionOperationManyDUnitTest.java |   28 +-
 ...StreamingPartitionOperationOneDUnitTest.java |   29 +-
 .../fixed/CustomerFixedPartitionResolver.java   |   21 +-
 .../fixed/FixedPartitioningDUnitTest.java       |   21 +-
 .../fixed/FixedPartitioningTestBase.java        |  110 +-
 ...ngWithColocationAndPersistenceDUnitTest.java |  127 +-
 .../cache/partitioned/fixed/MyDate1.java        |   21 +-
 .../cache/partitioned/fixed/MyDate2.java        |   21 +-
 .../cache/partitioned/fixed/MyDate3.java        |   21 +-
 .../fixed/QuarterPartitionResolver.java         |   21 +-
 .../SingleHopQuarterPartitionResolver.java      |   21 +-
 .../persistence/BackupInspectorJUnitTest.java   |   21 +-
 .../PersistentRVVRecoveryDUnitTest.java         |   21 +-
 .../PersistentRecoveryOrderDUnitTest.java       |   21 +-
 ...rsistentRecoveryOrderOldConfigDUnitTest.java |   21 +-
 .../PersistentReplicatedTestBase.java           |   21 +-
 .../TemporaryResultSetFactoryJUnitTest.java     |   21 +-
 .../cache/persistence/soplog/AppendLog.java     |   56 -
 .../ArraySerializedComparatorJUnitTest.java     |   86 -
 .../CompactionSortedOplogSetTestCase.java       |  125 -
 .../persistence/soplog/CompactionTestCase.java  |  197 -
 .../persistence/soplog/ComparisonTestCase.java  |   68 -
 .../soplog/IndexComparatorJUnitTest.java        |   70 -
 .../LexicographicalComparatorJUnitTest.java     |  195 -
 .../soplog/RecoverableSortedOplogSet.java       |  212 -
 .../soplog/SizeTieredCompactorJUnitTest.java    |  101 -
 .../SizeTieredSortedOplogSetJUnitTest.java      |   27 -
 .../soplog/SortedBufferJUnitTest.java           |   23 -
 .../soplog/SortedOplogSetJUnitTest.java         |  264 -
 .../soplog/SortedReaderTestCase.java            |  286 --
 .../nofile/NoFileSortedOplogJUnitTest.java      |   39 -
 .../GFSnapshotJUnitPerformanceTest.java         |   21 +-
 .../internal/cache/tier/Bug40396DUnitTest.java  |   21 +-
 .../tier/sockets/AcceptorImplJUnitTest.java     |   83 +-
 ...rdCompatibilityCommandDUnitDisabledTest.java |  235 -
 ...CompatibilityHandshakeDUnitDisabledTest.java |  218 -
 ...mpatibilityHigherVersionClientDUnitTest.java |   25 +-
 ...rdCompatibilityMessageDUnitDisabledTest.java |  299 --
 .../BridgeServerMaxConnectionsJUnitTest.java    |  221 -
 ...geServerSelectorMaxConnectionsJUnitTest.java |   19 -
 .../tier/sockets/BridgeWriterMiscDUnitTest.java | 1384 ------
 .../BridgeWriterMiscSelectorDUnitTest.java      |   27 -
 .../cache/tier/sockets/Bug36269DUnitTest.java   |   25 +-
 .../cache/tier/sockets/Bug36457DUnitTest.java   |   39 +-
 .../cache/tier/sockets/Bug36805DUnitTest.java   |   27 +-
 .../cache/tier/sockets/Bug36829DUnitTest.java   |   26 +-
 .../cache/tier/sockets/Bug36995DUnitTest.java   |   34 +-
 .../cache/tier/sockets/Bug37210DUnitTest.java   |   53 +-
 .../cache/tier/sockets/Bug37805DUnitTest.java   |   24 +-
 .../CacheServerMaxConnectionsJUnitTest.java     |  229 +
 ...heServerSelectorMaxConnectionsJUnitTest.java |   35 +
 .../cache/tier/sockets/CacheServerTestUtil.java |   82 +-
 .../CacheServerTransactionsDUnitTest.java       |   25 +-
 ...acheServerTransactionsSelectorDUnitTest.java |   16 +
 .../tier/sockets/ClearPropagationDUnitTest.java |   25 +-
 .../tier/sockets/ClientConflationDUnitTest.java |   41 +-
 .../sockets/ClientHealthMonitorJUnitTest.java   |   27 +-
 .../ClientHealthMonitorSelectorJUnitTest.java   |   16 +
 .../sockets/ClientInterestNotifyDUnitTest.java  |   35 +-
 .../tier/sockets/ClientServerMiscDUnitTest.java | 1392 ++++++
 .../ClientServerMiscSelectorDUnitTest.java      |   36 +
 .../cache/tier/sockets/ConflationDUnitTest.java |   45 +-
 .../tier/sockets/ConnectionProxyJUnitTest.java  |  278 +-
 .../DataSerializerPropogationDUnitTest.java     |   57 +-
 .../cache/tier/sockets/DeltaEOFException.java   |   21 +-
 .../DestroyEntryPropagationDUnitTest.java       |   33 +-
 .../sockets/DurableClientBug39997DUnitTest.java |   21 +-
 .../DurableClientQueueSizeDUnitTest.java        |   37 +-
 .../DurableClientReconnectAutoDUnitTest.java    |   21 +-
 .../DurableClientReconnectDUnitTest.java        |   33 +-
 .../sockets/DurableClientStatsDUnitTest.java    |   43 +-
 .../sockets/DurableRegistrationDUnitTest.java   |   63 +-
 .../sockets/DurableResponseMatrixDUnitTest.java |   31 +-
 .../sockets/EventIDVerificationDUnitTest.java   |   31 +-
 .../EventIDVerificationInP2PDUnitTest.java      |   20 +-
 .../cache/tier/sockets/FaultyDelta.java         |   21 +-
 .../tier/sockets/FilterProfileJUnitTest.java    |   21 +-
 .../ForceInvalidateEvictionDUnitTest.java       |   31 +-
 ...ForceInvalidateOffHeapEvictionDUnitTest.java |   21 +-
 .../cache/tier/sockets/HABug36738DUnitTest.java |   21 +-
 .../cache/tier/sockets/HAInterestBaseTest.java  | 1468 +++---
 .../sockets/HAInterestDistributedTestCase.java  |   31 +
 .../tier/sockets/HAInterestPart1DUnitTest.java  |  232 +-
 .../tier/sockets/HAInterestPart2DUnitTest.java  |  551 ++-
 .../sockets/HAStartupAndFailoverDUnitTest.java  |  102 +-
 .../internal/cache/tier/sockets/HaHelper.java   |   16 +
 .../InstantiatorPropagationDUnitTest.java       | 1777 +++++++
 .../tier/sockets/InterestListDUnitTest.java     |   25 +-
 .../sockets/InterestListEndpointDUnitTest.java  |   39 +-
 .../InterestListEndpointPRDUnitTest.java        |   21 +-
 .../InterestListEndpointSelectorDUnitTest.java  |   16 +
 .../sockets/InterestListFailoverDUnitTest.java  |   32 +-
 .../sockets/InterestListRecoveryDUnitTest.java  |   31 +-
 .../sockets/InterestRegrListenerDUnitTest.java  |   21 +-
 .../sockets/InterestResultPolicyDUnitTest.java  |   25 +-
 .../sockets/NewRegionAttributesDUnitTest.java   |   21 +-
 .../tier/sockets/ObjectPartListJUnitTest.java   |   21 +-
 .../tier/sockets/RedundancyLevelJUnitTest.java  |   34 +-
 .../sockets/RedundancyLevelPart1DUnitTest.java  |   25 +-
 .../sockets/RedundancyLevelPart2DUnitTest.java  |   21 +-
 .../sockets/RedundancyLevelPart3DUnitTest.java  |   27 +-
 .../tier/sockets/RedundancyLevelTestBase.java   |   72 +-
 .../tier/sockets/RegionCloseDUnitTest.java      |   35 +-
 ...erInterestBeforeRegionCreationDUnitTest.java |   27 +-
 .../sockets/RegisterInterestKeysDUnitTest.java  |   25 +-
 .../RegisterInterestKeysPRDUnitTest.java        |   21 +-
 .../sockets/ReliableMessagingDUnitTest.java     |   45 +-
 .../internal/cache/tier/sockets/TestPut.java    |   53 -
 .../sockets/UnregisterInterestDUnitTest.java    |   24 +-
 .../sockets/UpdatePropagationDUnitTest.java     |   31 +-
 .../sockets/UpdatePropagationPRDUnitTest.java   |   21 +-
 .../VerifyEventIDGenerationInP2PDUnitTest.java  |   20 +-
 ...UpdatesFromNonInterestEndPointDUnitTest.java |   25 +-
 .../tier/sockets/command/CommitCommandTest.java |   55 +
 .../cache/versions/RVVExceptionJUnitTest.java   |   21 +-
 .../versions/RegionVersionHolderJUnitTest.java  |   21 +-
 .../RegionVersionHolderRandomJUnitTest.java     |   21 +-
 ...RegionVersionHolderSmallBitSetJUnitTest.java |   21 +-
 .../versions/RegionVersionVectorJUnitTest.java  |  252 +-
 .../cache/wan/AsyncEventQueueTestBase.java      | 1666 +++++++
 .../cache/wan/CompressionConstants.java         |   21 +-
 .../cache/wan/CompressionInputStream.java       |   21 +-
 .../cache/wan/CompressionOutputStream.java      |   21 +-
 .../cache/wan/CustomAsyncEventListener.java     |   21 +-
 .../gemfire/internal/cache/wan/Filter70.java    |   21 +-
 .../cache/wan/MyAsyncEventListener.java         |   21 +-
 .../cache/wan/MyAsyncEventListener2.java        |   21 +-
 .../cache/wan/MyDistributedSystemListener.java  |   21 +-
 .../cache/wan/MyGatewaySenderEventListener.java |   21 +-
 .../wan/MyGatewaySenderEventListener2.java      |   21 +-
 .../cache/wan/MyGatewayTransportFilter1.java    |   21 +-
 .../cache/wan/MyGatewayTransportFilter2.java    |   21 +-
 .../cache/wan/MyGatewayTransportFilter3.java    |   21 +-
 .../cache/wan/MyGatewayTransportFilter4.java    |   21 +-
 .../internal/cache/wan/QueueListener.java       |   21 +-
 .../asyncqueue/AsyncEventListenerDUnitTest.java | 1920 ++++++++
 .../AsyncEventListenerOffHeapDUnitTest.java     |   33 +
 .../AsyncEventQueueStatsDUnitTest.java          |  320 ++
 .../AsyncEventQueueValidationsJUnitTest.java    |   21 +-
 .../ConcurrentAsyncEventQueueDUnitTest.java     |  336 ++
 ...ncurrentAsyncEventQueueOffHeapDUnitTest.java |   32 +
 .../CommonParallelAsyncEventQueueDUnitTest.java |   59 +
 ...ParallelAsyncEventQueueOffHeapDUnitTest.java |   32 +
 .../xmlcache/AbstractXmlParserJUnitTest.java    |   21 +-
 .../cache/xmlcache/CacheCreationJUnitTest.java  |  209 +
 .../cache/xmlcache/CacheXmlParserJUnitTest.java |   21 +-
 .../xmlcache/CacheXmlVersionJUnitTest.java      |   17 +-
 .../PivotalEntityResolverJUnitTest.java         |   21 +-
 .../cache/xmlcache/RegionCreationJUnitTest.java |   21 +-
 .../xmlcache/XmlGeneratorUtilsJUnitTest.java    |   21 +-
 .../classpathloaderjunittest/DoesExist.java     |   16 +
 .../CompressionCacheConfigDUnitTest.java        |   21 +-
 .../CompressionCacheListenerDUnitTest.java      |   21 +-
 ...ompressionCacheListenerOffHeapDUnitTest.java |   16 +
 .../CompressionRegionConfigDUnitTest.java       |   22 +-
 .../CompressionRegionFactoryDUnitTest.java      |   21 +-
 .../CompressionRegionOperationsDUnitTest.java   |   21 +-
 ...ressionRegionOperationsOffHeapDUnitTest.java |   16 +
 .../compression/CompressionStatsDUnitTest.java  |   21 +-
 .../compression/SnappyCompressorJUnitTest.java  |   25 +-
 .../datasource/AbstractPoolCacheJUnitTest.java  |   21 +-
 .../internal/datasource/CleanUpJUnitTest.java   |   21 +-
 .../ConnectionPoolCacheImplJUnitTest.java       |   21 +-
 .../datasource/ConnectionPoolingJUnitTest.java  |   21 +-
 .../datasource/DataSourceFactoryJUnitTest.java  |   21 +-
 .../internal/datasource/RestartJUnitTest.java   |   21 +-
 .../internal/i18n/BasicI18nJUnitTest.java       |   82 +-
 .../io/CompositeOutputStreamJUnitTest.java      |   21 +-
 .../gemfire/internal/jndi/ContextJUnitTest.java |   21 +-
 .../internal/jta/BlockingTimeOutJUnitTest.java  |   21 +-
 .../gemfire/internal/jta/CacheUtils.java        |   21 +-
 .../internal/jta/DataSourceJTAJUnitTest.java    |   21 +-
 .../internal/jta/ExceptionJUnitTest.java        |   21 +-
 .../jta/GlobalTransactionJUnitTest.java         |   21 +-
 .../gemstone/gemfire/internal/jta/JTAUtils.java |   21 +-
 .../internal/jta/JtaIntegrationJUnitTest.java   |   16 +
 .../gemstone/gemfire/internal/jta/SyncImpl.java |   21 +-
 .../internal/jta/TransactionImplJUnitTest.java  |   21 +-
 .../jta/TransactionManagerImplJUnitTest.java    |   21 +-
 .../jta/TransactionTimeOutJUnitTest.java        |   21 +-
 .../jta/UserTransactionImplJUnitTest.java       |   21 +-
 .../internal/jta/dunit/CommitThread.java        |   21 +-
 .../internal/jta/dunit/ExceptionsDUnitTest.java |   21 +-
 .../jta/dunit/IdleTimeOutDUnitTest.java         |   21 +-
 .../jta/dunit/LoginTimeOutDUnitTest.java        |   21 +-
 .../jta/dunit/MaxPoolSizeDUnitTest.java         |   21 +-
 .../internal/jta/dunit/RollbackThread.java      |   21 +-
 .../jta/dunit/TransactionTimeOutDUnitTest.java  |   21 +-
 .../dunit/TxnManagerMultiThreadDUnitTest.java   |   21 +-
 .../internal/jta/dunit/TxnTimeOutDUnitTest.java |   21 +-
 .../internal/jta/functional/CacheJUnitTest.java |   21 +-
 .../jta/functional/TestXACacheLoader.java       |   21 +-
 .../internal/lang/ClassUtilsJUnitTest.java      |   21 +-
 .../internal/lang/InOutParameterJUnitTest.java  |   21 +-
 .../internal/lang/InitializerJUnitTest.java     |   21 +-
 .../internal/lang/ObjectUtilsJUnitTest.java     |   21 +-
 .../internal/lang/StringUtilsJUnitTest.java     |   54 +-
 .../internal/lang/SystemUtilsJUnitTest.java     |   21 +-
 .../internal/lang/ThreadUtilsJUnitTest.java     |   58 +-
 .../DistributedSystemLogFileJUnitTest.java      |   36 +-
 .../logging/LocatorLogFileJUnitTest.java        |   16 +
 .../logging/LogServiceIntegrationJUnitTest.java |  223 +
 .../LogServiceIntegrationTestSupport.java       |   40 +
 .../internal/logging/LogServiceJUnitTest.java   |  204 +-
 .../LogWriterDisabledPerformanceTest.java       |   16 +
 .../logging/LogWriterImplJUnitTest.java         |   16 +
 .../logging/LogWriterPerformanceTest.java       |   16 +
 .../logging/LoggingIntegrationTestSuite.java    |   35 +
 .../logging/LoggingPerformanceTestCase.java     |   16 +
 .../internal/logging/LoggingUnitTestSuite.java  |   42 +
 .../logging/MergeLogFilesJUnitTest.java         |   25 +-
 .../gemfire/internal/logging/NullLogWriter.java |   16 +
 .../internal/logging/SortLogFileJUnitTest.java  |   21 +-
 .../internal/logging/TestLogWriterFactory.java  |   16 +
 .../logging/log4j/AlertAppenderJUnitTest.java   |   16 +
 .../logging/log4j/ConfigLocatorJUnitTest.java   |   16 +
 .../log4j/FastLoggerIntegrationJUnitTest.java   |  575 +++
 .../logging/log4j/FastLoggerJUnitTest.java      |  290 +-
 .../FastLoggerWithDefaultConfigJUnitTest.java   |   90 +
 .../log4j/LocalizedMessageJUnitTest.java        |   25 +-
 .../log4j/Log4J2DisabledPerformanceTest.java    |   16 +
 .../logging/log4j/Log4J2PerformanceTest.java    |   16 +
 .../log4j/Log4jIntegrationTestSuite.java        |   28 +
 .../logging/log4j/Log4jUnitTestSuite.java       |   32 +
 .../log4j/LogWriterAppenderJUnitTest.java       |   16 +
 .../LogWriterLoggerDisabledPerformanceTest.java |   16 +
 .../log4j/LogWriterLoggerPerformanceTest.java   |   16 +
 .../internal/net/SocketUtilsJUnitTest.java      |   21 +-
 .../offheap/AbstractStoredObjectTestBase.java   |  203 +
 .../offheap/ByteArrayMemoryChunkJUnitTest.java  |   16 +
 .../offheap/ChunkWithHeapFormJUnitTest.java     |   64 +
 .../offheap/ConcurrentBagJUnitTest.java         |  114 -
 .../offheap/DataAsAddressJUnitTest.java         |  368 ++
 .../internal/offheap/DataTypeJUnitTest.java     |   16 +
 .../DirectByteBufferMemoryChunkJUnitTest.java   |   16 +
 .../offheap/FreeListOffHeapRegionJUnitTest.java |   18 +-
 .../offheap/GemFireChunkFactoryJUnitTest.java   |  129 +
 .../internal/offheap/GemFireChunkJUnitTest.java |  921 ++++
 .../offheap/GemFireChunkSliceJUnitTest.java     |   72 +
 .../HeapByteBufferMemoryChunkJUnitTest.java     |   16 +
 .../internal/offheap/InlineKeyJUnitTest.java    |   16 +
 .../offheap/LifecycleListenerJUnitTest.java     |  222 +
 .../offheap/MemoryChunkJUnitTestBase.java       |   63 +
 .../offheap/NullOffHeapMemoryStats.java         |   22 +
 .../offheap/NullOutOfOffHeapMemoryListener.java |   22 +
 .../internal/offheap/OffHeapIndexJUnitTest.java |   16 +
 .../internal/offheap/OffHeapRegionBase.java     |   17 +-
 .../offheap/OffHeapStorageJUnitTest.java        |  218 +
 .../offheap/OffHeapValidationJUnitTest.java     |   62 +-
 .../OffHeapWriteObjectAsByteArrayJUnitTest.java |  115 +
 .../OldFreeListOffHeapRegionJUnitTest.java      |   18 +-
 .../offheap/OutOfOffHeapMemoryDUnitTest.java    |   43 +-
 .../offheap/RefCountChangeInfoJUnitTest.java    |  207 +
 ...mpleMemoryAllocatorFillPatternJUnitTest.java |   18 +-
 .../offheap/SimpleMemoryAllocatorJUnitTest.java |  394 +-
 ...moryAllocatorLifecycleListenerJUnitTest.java |  132 -
 .../internal/offheap/StoredObjectTestSuite.java |   33 +
 .../TxReleasesOffHeapOnCloseJUnitTest.java      |   16 +
 .../offheap/UnsafeMemoryChunkJUnitTest.java     |   16 +
 .../BlockingProcessStreamReaderJUnitTest.java   |   16 +
 .../LocalProcessControllerJUnitTest.java        |   21 +-
 .../process/LocalProcessLauncherDUnitTest.java  |   21 +-
 .../process/LocalProcessLauncherJUnitTest.java  |   21 +-
 ...NonBlockingProcessStreamReaderJUnitTest.java |   16 +
 .../internal/process/PidFileJUnitTest.java      |   16 +
 .../ProcessControllerFactoryJUnitTest.java      |   16 +
 .../process/ProcessStreamReaderTestCase.java    |   22 +-
 .../gemfire/internal/process/mbean/Process.java |   16 +
 .../internal/process/mbean/ProcessMBean.java    |   16 +
 ...tractSignalNotificationHandlerJUnitTest.java |   21 +-
 .../internal/size/ObjectSizerJUnitTest.java     |   35 +-
 .../internal/size/ObjectTraverserJUnitTest.java |   21 +-
 .../internal/size/ObjectTraverserPerf.java      |   21 +-
 .../size/SizeClassOnceObjectSizerJUnitTest.java |   21 +-
 .../gemfire/internal/size/SizeTestUtil.java     |   16 +
 .../size/WellKnownClassSizerJUnitTest.java      |   21 +-
 .../internal/statistics/DummyStatistics.java    |   21 +-
 .../statistics/SampleCollectorJUnitTest.java    |   21 +-
 .../statistics/StatMonitorHandlerJUnitTest.java |   21 +-
 .../statistics/StatisticsDUnitTest.java         |   21 +-
 .../statistics/StatisticsMonitorJUnitTest.java  |   21 +-
 .../internal/statistics/TestSampleHandler.java  |   21 +-
 .../statistics/TestStatArchiveWriter.java       |   21 +-
 .../statistics/TestStatisticsManager.java       |   21 +-
 .../statistics/TestStatisticsSampler.java       |   21 +-
 .../statistics/ValueMonitorJUnitTest.java       |   21 +-
 .../internal/stats50/AtomicStatsJUnitTest.java  |   37 +-
 .../internal/tcp/ConnectionJUnitTest.java       |   88 +
 .../util/AbortableTaskServiceJUnitTest.java     |   21 +-
 .../internal/util/ArrayUtilsJUnitTest.java      |   18 +-
 .../gemfire/internal/util/BytesJUnitTest.java   |   21 +-
 .../internal/util/CollectionUtilsJUnitTest.java |   18 +-
 .../internal/util/DelayedActionJUnitTest.java   |   21 +-
 .../gemfire/internal/util/IOUtilsJUnitTest.java |   21 +-
 .../gemfire/internal/util/SerializableImpl.java |   21 +-
 .../util/SerializableImplWithValue.java         |   21 +-
 .../gemfire/internal/util/Valuable.java         |   21 +-
 .../CompactConcurrentHashSetJUnitTest.java      |   16 +
 .../ConcurrentHashMapIteratorJUnitTest.java     |   21 +-
 .../concurrent/CopyOnWriteHashMapJUnitTest.java |  505 ++
 .../concurrent/ReentrantSemaphoreJUnitTest.java |   22 +-
 .../SemaphoreReadWriteLockJUnitTest.java        |   21 +-
 .../cm/ConcurrentHashMapJUnitTest.java          |   21 +-
 .../concurrent/cm/CountedMapLoopsJUnitTest.java |   21 +-
 .../concurrent/cm/IntMapCheckJUnitTest.java     |   21 +-
 .../util/concurrent/cm/LoopHelpers.java         |   21 +-
 .../util/concurrent/cm/MapCheckJUnitTest.java   |   21 +-
 .../util/concurrent/cm/MapLoopsJUnitTest.java   |   21 +-
 .../util/concurrent/cm/RLJBarJUnitTest.java     |   21 +-
 .../concurrent/cm/StringMapLoopsJUnitTest.java  |   21 +-
 .../management/CacheManagementDUnitTest.java    |   33 +-
 .../management/ClientHealthStatsDUnitTest.java  |   23 +-
 .../gemfire/management/CompositeStats.java      |   21 +-
 .../gemfire/management/CompositeTestMBean.java  |   21 +-
 .../gemfire/management/CompositeTestMXBean.java |   16 +
 .../management/CompositeTypeTestDUnitTest.java  |   21 +-
 .../gemfire/management/CustomMBean.java         |   22 +-
 .../gemfire/management/CustomMXBean.java        |   20 +-
 .../management/DLockManagementDUnitTest.java    |   20 +-
 .../DataBrowserJSONValidationJUnitTest.java     |   21 +-
 .../management/DiskManagementDUnitTest.java     |   20 +-
 .../management/DistributedSystemDUnitTest.java  |   81 +-
 .../management/LocatorManagementDUnitTest.java  |   25 +-
 .../gemstone/gemfire/management/MBeanUtil.java  |   20 +-
 .../gemfire/management/ManagementTestBase.java  |   21 +-
 .../MemberMBeanAttributesDUnitTest.java         |   20 +-
 .../management/OffHeapManagementDUnitTest.java  |   32 +-
 .../gemfire/management/QueryDataDUnitTest.java  |   21 +-
 .../management/RegionManagementDUnitTest.java   |   20 +-
 .../gemfire/management/TypedJsonJUnitTest.java  |   21 +-
 ...ersalMembershipListenerAdapterDUnitTest.java |   53 +-
 .../stats/AsyncEventQueueStatsJUnitTest.java    |   21 +-
 .../bean/stats/CacheServerStatsJUnitTest.java   |   21 +-
 .../bean/stats/DiskStatsJUnitTest.java          |   21 +-
 .../stats/DistributedSystemStatsDUnitTest.java  |   21 +-
 .../stats/DistributedSystemStatsJUnitTest.java  |   21 +-
 .../stats/GatewayReceiverStatsJUnitTest.java    |   21 +-
 .../bean/stats/GatewaySenderStatsJUnitTest.java |   21 +-
 .../HDFSRegionMBeanAttributeJUnitTest.java      |  169 -
 .../bean/stats/MBeanStatsTestCase.java          |   21 +-
 .../bean/stats/MemberLevelStatsJUnitTest.java   |   21 +-
 .../bean/stats/RegionStatsJUnitTest.java        |   21 +-
 .../bean/stats/StatsRateJUnitTest.java          |   21 +-
 .../internal/JettyHelperJUnitTest.java          |   21 +-
 .../cli/ClasspathScanLoadHelperJUnitTest.java   |   21 +-
 .../internal/cli/CliUtilDUnitTest.java          |   21 +-
 .../internal/cli/CommandManagerJUnitTest.java   |   21 +-
 .../cli/CommandSeparatorEscapeJUnitTest.java    |   16 +
 .../internal/cli/DataCommandJsonJUnitTest.java  |   16 +
 .../internal/cli/GfshParserJUnitTest.java       |   69 +-
 .../management/internal/cli/HeadlessGfsh.java   |  376 ++
 .../internal/cli/HeadlessGfshJUnitTest.java     |   86 +
 .../management/internal/cli/ResultHandler.java  |   23 +
 .../internal/cli/TableBuilderJUnitTest.java     |  183 +
 .../cli/annotations/CliArgumentJUnitTest.java   |   21 +-
 .../AbstractCommandsSupportJUnitTest.java       |   21 +-
 .../cli/commands/CliCommandTestBase.java        |  560 +++
 .../cli/commands/ConfigCommandsDUnitTest.java   |  497 ++
 ...eateAlterDestroyRegionCommandsDUnitTest.java | 1148 +++++
 .../cli/commands/DeployCommandsDUnitTest.java   |  480 ++
 .../commands/DiskStoreCommandsDUnitTest.java    | 1154 +++++
 .../commands/DiskStoreCommandsJUnitTest.java    |   21 +-
 .../cli/commands/FunctionCommandsDUnitTest.java |  593 +++
 .../commands/GemfireDataCommandsDUnitTest.java  | 2087 ++++++++
 ...WithCacheLoaderDuringCacheMissDUnitTest.java |  371 ++
 .../commands/HDFSStoreCommandsJUnitTest.java    |  878 ----
 .../HTTPServiceSSLSupportJUnitTest.java         |   20 +-
 .../cli/commands/IndexCommandsDUnitTest.java    |  817 ++++
 .../cli/commands/IndexCommandsJUnitTest.java    |   21 +-
 ...stAndDescribeDiskStoreCommandsDUnitTest.java |  197 +
 .../ListAndDescribeRegionDUnitTest.java         |  320 ++
 .../cli/commands/ListIndexCommandDUnitTest.java |  672 +++
 .../cli/commands/MemberCommandsDUnitTest.java   |  287 ++
 .../MiscellaneousCommandsDUnitTest.java         |  492 ++
 ...laneousCommandsExportLogsPart1DUnitTest.java |  139 +
 ...laneousCommandsExportLogsPart2DUnitTest.java |  148 +
 ...laneousCommandsExportLogsPart3DUnitTest.java |  150 +
 ...laneousCommandsExportLogsPart4DUnitTest.java |  141 +
 .../cli/commands/QueueCommandsDUnitTest.java    |  385 ++
 .../SharedConfigurationCommandsDUnitTest.java   |  338 ++
 .../cli/commands/ShellCommandsDUnitTest.java    |  365 ++
 .../cli/commands/ShowDeadlockDUnitTest.java     |  271 ++
 .../cli/commands/ShowMetricsDUnitTest.java      |  347 ++
 .../cli/commands/ShowStackTraceDUnitTest.java   |  149 +
 .../cli/commands/UserCommandsDUnitTest.java     |  164 +
 .../RegionPathConverterJUnitTest.java           |   21 +-
 .../internal/cli/domain/AbstractImpl.java       |   16 +
 .../management/internal/cli/domain/Impl1.java   |   16 +
 .../management/internal/cli/domain/Impl12.java  |   16 +
 .../internal/cli/domain/Interface1.java         |   16 +
 .../internal/cli/domain/Interface2.java         |   16 +
 .../management/internal/cli/domain/Stock.java   |   21 +-
 .../management/internal/cli/dto/Car.java        |   16 +
 .../management/internal/cli/dto/Key1.java       |   21 +-
 .../management/internal/cli/dto/Key2.java       |   21 +-
 .../internal/cli/dto/ObjectWithCharAttr.java    |   21 +-
 .../management/internal/cli/dto/Value1.java     |   21 +-
 .../management/internal/cli/dto/Value2.java     |   21 +-
 .../AlterHDFSStoreFunctionJUnitTest.java        |  462 --
 .../CreateHDFSStoreFunctionJUnitTest.java       |  429 --
 .../functions/DataCommandFunctionJUnitTest.java |  132 +
 .../DescribeDiskStoreFunctionJUnitTest.java     |   21 +-
 .../DescribeHDFSStoreFunctionJUnitTest.java     |  396 --
 .../DestroyHDFSStoreFunctionJUnitTest.java      |  305 --
 .../ListDiskStoresFunctionJUnitTest.java        |   21 +-
 .../ListHDFSStoresFunctionJUnitTest.java        |  319 --
 .../functions/ListIndexFunctionJUnitTest.java   |   21 +-
 .../cli/parser/ParserUtilsJUnitTest.java        |   21 +-
 .../preprocessor/PreprocessorJUnitTest.java     |   21 +-
 .../PreprocessorUtilsJUnitTest.java             |   21 +-
 .../cli/shell/GfshConfigInitFileJUnitTest.java  |   16 +
 .../shell/GfshExecutionStrategyJUnitTest.java   |   21 +-
 .../cli/shell/GfshInitFileJUnitTest.java        |   16 +
 .../SharedConfigurationDUnitTest.java           |   21 +-
 .../configuration/ZipUtilsJUnitTest.java        |   21 +-
 .../domain/CacheElementJUnitTest.java           |   21 +-
 .../utils/XmlUtilsAddNewNodeJUnitTest.java      |   21 +-
 .../configuration/utils/XmlUtilsJUnitTest.java  |   21 +-
 .../internal/pulse/TestClientIdsDUnitTest.java  |   22 +-
 .../internal/pulse/TestFunctionsDUnitTest.java  |   22 +-
 .../internal/pulse/TestHeapDUnitTest.java       |   23 +-
 .../internal/pulse/TestLocatorsDUnitTest.java   |   22 +-
 .../pulse/TestSubscriptionsDUnitTest.java       |   20 +-
 .../internal/security/JSONAuthCodeTest.java     |   16 +
 .../security/JSONAuthorizationTest.java         |   16 +
 .../security/ResourceOperationJUnit.java        |   16 +
 .../ReadOpFileAccessControllerJUnitTest.java    |   21 +-
 .../WanCommandsControllerJUnitTest.java         |  140 +
 .../gemfire/management/model/EmptyObject.java   |   21 +-
 .../gemstone/gemfire/management/model/Item.java |   21 +-
 .../gemfire/management/model/Order.java         |   21 +-
 .../gemfire/management/model/SubOrder.java      |   21 +-
 .../DomainObjectsAsValuesJUnitTest.java         |   21 +-
 .../GemcachedBinaryClientJUnitTest.java         |   21 +-
 .../GemcachedDevelopmentJUnitTest.java          |   21 +-
 .../gemfire/memcached/IntegrationJUnitTest.java |   24 +-
 .../gemfire/pdx/AutoSerializableJUnitTest.java  |   23 +-
 .../gemfire/pdx/ByteSourceJUnitTest.java        |   20 +-
 .../ClientsWithVersioningRetryDUnitTest.java    |   21 +-
 .../com/gemstone/gemfire/pdx/DSInsidePdx.java   |   23 +-
 .../pdx/DistributedSystemIdDUnitTest.java       |   21 +-
 .../com/gemstone/gemfire/pdx/DomainObject.java  |   21 +-
 .../gemstone/gemfire/pdx/DomainObjectBad.java   |   16 +
 .../gemfire/pdx/DomainObjectClassLoadable.java  |   16 +
 .../gemfire/pdx/DomainObjectPdxAuto.java        |   21 +-
 ...DomainObjectPdxAutoNoDefaultConstructor.java |   21 +-
 .../java/com/gemstone/gemfire/pdx/Employee.java |   23 +-
 .../pdx/JSONPdxClientServerDUnitTest.java       |   23 +-
 .../com/gemstone/gemfire/pdx/NestedPdx.java     |   21 +-
 .../gemfire/pdx/NonDelegatingLoader.java        |   23 +-
 .../OffHeapByteBufferByteSourceJUnitTest.java   |   18 +-
 .../gemfire/pdx/OffHeapByteSourceJUnitTest.java |   18 +-
 .../pdx/PDXAsyncEventQueueDUnitTest.java        |   22 +-
 .../gemfire/pdx/PdxAttributesJUnitTest.java     |   21 +-
 .../gemfire/pdx/PdxClientServerDUnitTest.java   |   21 +-
 .../pdx/PdxDeserializationDUnitTest.java        |   21 +-
 .../pdx/PdxFormatterPutGetJUnitTest.java        |   25 +-
 .../com/gemstone/gemfire/pdx/PdxInsideDS.java   |   23 +-
 .../pdx/PdxInstanceFactoryJUnitTest.java        |   21 +-
 .../gemfire/pdx/PdxInstanceJUnitTest.java       |   21 +-
 .../gemfire/pdx/PdxSerializableDUnitTest.java   |   21 +-
 .../gemfire/pdx/PdxSerializableJUnitTest.java   |   21 +-
 .../gemfire/pdx/PdxStringJUnitTest.java         |   21 +-
 .../gemfire/pdx/PdxTypeExportDUnitTest.java     |   21 +-
 .../gemfire/pdx/SeparateClassloaderPdx.java     |   23 +-
 .../com/gemstone/gemfire/pdx/SimpleClass.java   |   23 +-
 .../com/gemstone/gemfire/pdx/SimpleClass1.java  |   21 +-
 .../com/gemstone/gemfire/pdx/SimpleClass2.java  |   21 +-
 .../gemfire/pdx/TestObjectForPdxFormatter.java  |   21 +-
 .../gemfire/pdx/VersionClassLoader.java         |   22 +-
 .../gemstone/gemfire/redis/AuthJUnitTest.java   |  160 +
 .../gemfire/redis/ConcurrentStartTest.java      |   74 +
 .../gemstone/gemfire/redis/HashesJUnitTest.java |  191 +
 .../gemstone/gemfire/redis/ListsJUnitTest.java  |  254 +
 .../gemfire/redis/RedisDistDUnitTest.java       |  257 +
 .../gemstone/gemfire/redis/SetsJUnitTest.java   |  258 +
 .../gemfire/redis/SortedSetsJUnitTest.java      |  430 ++
 .../gemfire/redis/StringsJunitTest.java         |  312 ++
 .../web/controllers/AddFreeItemToOrders.java    |   21 +-
 .../rest/internal/web/controllers/Customer.java |   21 +-
 .../internal/web/controllers/DateTimeUtils.java |   16 +
 .../rest/internal/web/controllers/Gender.java   |   16 +
 .../internal/web/controllers/GetAllEntries.java |   21 +-
 .../web/controllers/GetDeliveredOrders.java     |   21 +-
 .../internal/web/controllers/GetRegions.java    |   21 +-
 .../web/controllers/GetValueForKey.java         |   21 +-
 .../rest/internal/web/controllers/Item.java     |   21 +-
 .../rest/internal/web/controllers/Order.java    |   21 +-
 .../rest/internal/web/controllers/Person.java   |   21 +-
 .../web/controllers/PutKeyFunction.java         |   21 +-
 .../web/controllers/RestAPITestBase.java        |   16 +
 .../internal/web/controllers/RestTestUtils.java |   21 +-
 .../gemfire/test/golden/ExecutableProcess.java  |   24 +
 .../gemfire/test/golden/FailOutputTestCase.java |   38 +-
 .../golden/FailWithErrorInOutputJUnitTest.java  |   34 +-
 .../FailWithExtraLineInOutputJUnitTest.java     |   57 +-
 ...WithLineMissingFromEndOfOutputJUnitTest.java |   55 +-
 ...hLineMissingFromMiddleOfOutputJUnitTest.java |   55 +-
 .../FailWithLoggerErrorInOutputJUnitTest.java   |   34 +-
 .../FailWithLoggerFatalInOutputJUnitTest.java   |   34 +-
 .../FailWithLoggerWarnInOutputJUnitTest.java    |   34 +-
 .../golden/FailWithProblemInOutputTestCase.java |   46 +-
 .../golden/FailWithSevereInOutputJUnitTest.java |   34 +-
 ...hTimeoutOfWaitForOutputToMatchJUnitTest.java |   58 +-
 .../FailWithWarningInOutputJUnitTest.java       |   34 +-
 .../gemfire/test/golden/GoldenComparator.java   |   80 +-
 .../test/golden/GoldenStringComparator.java     |   21 +-
 .../gemfire/test/golden/GoldenTestCase.java     |  124 +-
 .../golden/GoldenTestFrameworkTestSuite.java    |   43 +
 .../gemfire/test/golden/PassJUnitTest.java      |   66 +-
 .../golden/PassWithExpectedErrorJUnitTest.java  |   34 +-
 .../golden/PassWithExpectedProblemTestCase.java |   74 +-
 .../golden/PassWithExpectedSevereJUnitTest.java |   34 +-
 .../PassWithExpectedWarningJUnitTest.java       |   34 +-
 .../test/golden/RegexGoldenComparator.java      |   26 +-
 .../test/golden/StringGoldenComparator.java     |   20 +-
 .../gemfire/test/process/MainLauncher.java      |   16 +
 .../test/process/MainLauncherJUnitTest.java     |   16 +
 .../gemfire/test/process/OutputFormatter.java   |   16 +
 .../test/process/ProcessOutputReader.java       |  117 +-
 .../test/process/ProcessStreamReader.java       |   58 +-
 .../process/ProcessTestFrameworkTestSuite.java  |   28 +
 .../gemfire/test/process/ProcessWrapper.java    |  271 +-
 .../test/process/ProcessWrapperJUnitTest.java   |   35 +-
 .../gemstone/gemfire/util/JSR166TestCase.java   |   22 +-
 .../gemstone/gemfire/util/test/TestUtil.java    |   16 +
 .../protocols/CacheTimeSlowDownDUnitTest.java   |  204 -
 .../GemFireTimeSyncProtocolDUnitTest.java       |  477 --
 .../JGroupsFailureDetectionJUnitTest.java       |  488 --
 .../protocols/JGroupsVersioningJUnitTest.java   |  343 --
 .../com/gemstone/persistence/admin/Logger.java  |   21 +-
 .../gemstone/persistence/logging/Formatter.java |   21 +-
 .../gemstone/persistence/logging/Handler.java   |   21 +-
 .../com/gemstone/persistence/logging/Level.java |   21 +-
 .../gemstone/persistence/logging/LogRecord.java |   21 +-
 .../gemstone/persistence/logging/Logger.java    |   21 +-
 .../persistence/logging/SimpleFormatter.java    |   21 +-
 .../persistence/logging/StreamHandler.java      |   21 +-
 .../test/java/com/gemstone/sequence/Arrow.java  |   21 +-
 .../java/com/gemstone/sequence/Lifeline.java    |   21 +-
 .../com/gemstone/sequence/LifelineState.java    |   21 +-
 .../java/com/gemstone/sequence/LineMapper.java  |   21 +-
 .../com/gemstone/sequence/SequenceDiagram.java  |   21 +-
 .../com/gemstone/sequence/SequencePanel.java    |   21 +-
 .../com/gemstone/sequence/StateColorMap.java    |   21 +-
 .../java/com/gemstone/sequence/TimeAxis.java    |   21 +-
 .../com/gemstone/sequence/ZoomingPanel.java     |   21 +-
 .../sequence/gemfire/DefaultLineMapper.java     |   21 +-
 .../gemfire/GemfireSequenceDisplay.java         |   21 +-
 .../sequence/gemfire/HydraLineMapper.java       |   21 +-
 .../sequence/gemfire/SelectGraphDialog.java     |   21 +-
 .../com/main/MyDistributedSystemListener.java   |   21 +-
 .../com/main/WANBootStrapping_Site1_Add.java    |   21 +-
 .../com/main/WANBootStrapping_Site1_Remove.java |   21 +-
 .../com/main/WANBootStrapping_Site2_Add.java    |   21 +-
 .../com/main/WANBootStrapping_Site2_Remove.java |   21 +-
 .../src/test/java/dunit/AsyncInvocation.java    |   21 +-
 .../src/test/java/dunit/BounceResult.java       |   16 +
 gemfire-core/src/test/java/dunit/DUnitEnv.java  |   21 +-
 .../test/java/dunit/DistributedTestCase.java    |  174 +-
 gemfire-core/src/test/java/dunit/Host.java      |   21 +-
 .../src/test/java/dunit/RMIException.java       |   21 +-
 .../src/test/java/dunit/RemoteDUnitVMIF.java    |   18 +
 .../src/test/java/dunit/RepeatableRunnable.java |   16 +
 .../test/java/dunit/SerializableCallable.java   |   21 +-
 .../test/java/dunit/SerializableRunnable.java   |   21 +-
 gemfire-core/src/test/java/dunit/VM.java        |   21 +-
 .../src/test/java/dunit/standalone/ChildVM.java |   36 +-
 .../java/dunit/standalone/DUnitLauncher.java    |   73 +-
 .../java/dunit/standalone/ProcessManager.java   |   62 +-
 .../java/dunit/standalone/RemoteDUnitVM.java    |   28 +-
 .../dunit/standalone/StandAloneDUnitEnv.java    |   21 +-
 .../test/java/dunit/tests/BasicDUnitTest.java   |   21 +-
 .../src/test/java/dunit/tests/TestFailure.java  |   21 +-
 .../src/test/java/dunit/tests/VMDUnitTest.java  |   21 +-
 gemfire-core/src/test/java/hydra/GsRandom.java  |   21 +-
 .../test/java/hydra/HydraRuntimeException.java  |   21 +-
 gemfire-core/src/test/java/hydra/Log.java       |   21 +-
 .../src/test/java/hydra/LogVersionHelper.java   |   21 +-
 .../src/test/java/hydra/MethExecutor.java       |   21 +-
 .../src/test/java/hydra/MethExecutorResult.java |   21 +-
 .../src/test/java/hydra/SchedulingOrder.java    |   21 +-
 .../src/test/java/hydra/log/AnyLogWriter.java   |   23 +-
 .../java/hydra/log/CircularOutputStream.java    |   21 +-
 .../parReg/query/unittest/NewPortfolio.java     |   21 +-
 .../java/parReg/query/unittest/Position.java    |   22 +-
 .../src/test/java/perffmwk/Formatter.java       |   22 +-
 .../templates/security/DummyAuthenticator.java  |   21 +-
 .../templates/security/DummyAuthorization.java  |   21 +-
 .../security/FunctionSecurityPrmsHolder.java    |   21 +-
 .../security/LdapUserAuthenticator.java         |   21 +-
 .../java/templates/security/PKCSAuthInit.java   |   21 +-
 .../templates/security/PKCSAuthenticator.java   |   21 +-
 .../java/templates/security/PKCSPrincipal.java  |   21 +-
 .../security/UserPasswordAuthInit.java          |   21 +-
 .../templates/security/UsernamePrincipal.java   |   21 +-
 .../templates/security/XmlAuthorization.java    |   21 +-
 .../templates/security/XmlErrorHandler.java     |   21 +-
 .../src/test/java/util/TestException.java       |   21 +-
 ...gemstone.gemfire.internal.cache.CacheService |    1 +
 .../ClientCacheFactoryJUnitTest_single_pool.xml |   17 +
 .../cache/client/internal/cacheserver.cer       |  Bin 0 -> 782 bytes
 .../cache/client/internal/cacheserver.keystore  |  Bin 0 -> 1253 bytes
 .../client/internal/cacheserver.truststore      |  Bin 0 -> 844 bytes
 .../gemfire/cache/client/internal/client.cer    |  Bin 0 -> 782 bytes
 .../cache/client/internal/client.keystore       |  Bin 0 -> 1251 bytes
 .../cache/client/internal/client.truststore     |  Bin 0 -> 846 bytes
 .../cache/client/internal/default.keystore      |  Bin 0 -> 1115 bytes
 .../cache/client/internal/trusted.keystore      |  Bin 0 -> 1078 bytes
 .../gemfire/cache/query/dunit/IndexCreation.xml |   17 +
 .../functional/index-creation-with-eviction.xml |   17 +
 .../index-creation-without-eviction.xml         |   17 +
 .../functional/index-recovery-overflow.xml      |   17 +
 .../query/internal/index/cachequeryindex.xml    |   17 +
 .../internal/index/cachequeryindexwitherror.xml |   17 +
 .../cache/query/partitioned/PRIndexCreation.xml |   17 +
 .../gemfire/cache30/attributesUnordered.xml     |   17 +
 .../com/gemstone/gemfire/cache30/badFloat.xml   |   17 +
 .../com/gemstone/gemfire/cache30/badInt.xml     |   17 +
 .../gemfire/cache30/badKeyConstraintClass.xml   |   17 +
 .../com/gemstone/gemfire/cache30/badScope.xml   |   17 +
 .../com/gemstone/gemfire/cache30/bug44710.xml   |   17 +
 .../gemfire/cache30/callbackNotDeclarable.xml   |   17 +
 .../gemfire/cache30/callbackWithException.xml   |   17 +
 .../com/gemstone/gemfire/cache30/coLocation.xml |   17 +
 .../gemstone/gemfire/cache30/coLocation3.xml    |   17 +
 .../com/gemstone/gemfire/cache30/ewtest.xml     |   17 +
 .../cache30/examples_3_0/example-cache.xml      |   16 +
 .../cache30/examples_4_0/example-cache.xml      |   16 +
 .../gemfire/cache30/loaderNotLoader.xml         |   17 +
 .../com/gemstone/gemfire/cache30/malformed.xml  |   17 +
 .../gemfire/cache30/namedAttributes.xml         |   17 +
 .../gemfire/cache30/partitionedRegion.xml       |   17 +
 .../gemfire/cache30/partitionedRegion51.xml     |   17 +
 .../gemstone/gemfire/cache30/sameRootRegion.xml |   17 +
 .../gemstone/gemfire/cache30/sameSubregion.xml  |   17 +
 .../gemfire/cache30/unknownNamedAttributes.xml  |   17 +
 .../gemfire/codeAnalysis/excludedClasses.txt    |    3 +-
 .../sanctionedDataSerializables.txt             | 1839 ++++---
 .../codeAnalysis/sanctionedSerializables.txt    |  165 +-
 .../internal/SharedConfigurationJUnitTest.xml   |   17 +
 .../internal/cache/BackupJUnitTest.cache.xml    |   17 +
 .../internal/cache/DiskRegCacheXmlJUnitTest.xml |   16 +
 .../cache/PartitionRegionCacheExample1.xml      |   17 +
 .../cache/PartitionRegionCacheExample2.xml      |   17 +
 .../incorrect_bytes_threshold.xml               |   17 +
 .../faultyDiskXMLsForTesting/incorrect_dir.xml  |   17 +
 .../incorrect_dir_size.xml                      |   17 +
 .../incorrect_max_oplog_size.xml                |   17 +
 .../incorrect_roll_oplogs_value.xml             |   17 +
 .../incorrect_sync_value.xml                    |   17 +
 .../incorrect_time_interval.xml                 |   17 +
 .../mixed_diskstore_diskdir.xml                 |   17 +
 .../mixed_diskstore_diskwriteattrs.xml          |   17 +
 .../tier/sockets/RedundancyLevelJUnitTest.xml   |   37 +-
 ...testDTDFallbackWithNonEnglishLocal.cache.xml |   17 +
 .../gemstone/gemfire/internal/jta/cachejta.xml  |   17 +
 .../gemfire/test/golden/log4j2-test.xml         |   18 +
 .../src/test/resources/jta/cachejta.xml         |   17 +
 .../resources/spring/spring-gemfire-context.xml |   17 +
 .../resources/templates/security/authz5_5.dtd   |   16 +
 .../resources/templates/security/authz6_0.dtd   |   16 +
 gemfire-jgroups/build.gradle                    |   39 -
 .../java/com/gemstone/org/jgroups/Address.java  |   46 -
 .../com/gemstone/org/jgroups/BlockEvent.java    |   15 -
 .../java/com/gemstone/org/jgroups/Channel.java  |  508 --
 .../org/jgroups/ChannelClosedException.java     |   27 -
 .../gemstone/org/jgroups/ChannelException.java  |   28 -
 .../gemstone/org/jgroups/ChannelFactory.java    |   38 -
 .../gemstone/org/jgroups/ChannelListener.java   |   20 -
 .../jgroups/ChannelNotConnectedException.java   |   26 -
 .../java/com/gemstone/org/jgroups/Event.java    |  225 -
 .../com/gemstone/org/jgroups/ExitEvent.java     |   14 -
 .../com/gemstone/org/jgroups/GetStateEvent.java |   22 -
 .../java/com/gemstone/org/jgroups/Global.java   |   44 -
 .../java/com/gemstone/org/jgroups/Header.java   |   51 -
 .../java/com/gemstone/org/jgroups/JChannel.java | 1725 -------
 .../gemstone/org/jgroups/JChannelFactory.java   |  129 -
 .../gemstone/org/jgroups/JGroupsVersion.java    |   95 -
 .../com/gemstone/org/jgroups/Membership.java    |  358 --
 .../org/jgroups/MembershipListener.java         |   53 -
 .../com/gemstone/org/jgroups/MergeView.java     |  166 -
 .../java/com/gemstone/org/jgroups/Message.java  |  784 ---
 .../com/gemstone/org/jgroups/Message.java.old   |  711 ---
 .../gemstone/org/jgroups/MessageListener.java   |   34 -
 .../java/com/gemstone/org/jgroups/Receiver.java |   13 -
 .../gemstone/org/jgroups/ReceiverAdapter.java   |   35 -
 .../com/gemstone/org/jgroups/SetStateEvent.java |   37 -
 .../org/jgroups/ShunnedAddressException.java    |   14 -
 .../com/gemstone/org/jgroups/SuspectEvent.java  |   26 -
 .../com/gemstone/org/jgroups/SuspectMember.java |   42 -
 .../org/jgroups/SuspectedException.java         |   21 -
 .../gemstone/org/jgroups/TimeoutException.java  |   43 -
 .../com/gemstone/org/jgroups/Transport.java     |   22 -
 .../com/gemstone/org/jgroups/UpHandler.java     |   18 -
 .../java/com/gemstone/org/jgroups/View.java     |  528 --
 .../java/com/gemstone/org/jgroups/ViewId.java   |  174 -
 .../org/jgroups/blocks/ConnectionTable.java     | 1051 ----
 .../org/jgroups/blocks/ConnectionTableNIO.java  | 1519 ------
 .../jgroups/blocks/DistributedHashtable.java    |  656 ---
 .../jgroups/blocks/DistributedLockManager.java  |  742 ---
 .../org/jgroups/blocks/DistributedQueue.java    |  758 ---
 .../org/jgroups/blocks/DistributedTree.java     |  756 ---
 .../org/jgroups/blocks/GroupRequest.java        |  636 ---
 .../org/jgroups/blocks/GroupRequest.java.old    |  641 ---
 .../com/gemstone/org/jgroups/blocks/Link.java   |  680 ---
 .../com/gemstone/org/jgroups/blocks/Link.txt    |   48 -
 .../org/jgroups/blocks/LockManager.java         |   87 -
 .../blocks/LockMultiLockedException.java        |   26 -
 .../jgroups/blocks/LockNotGrantedException.java |   24 -
 .../blocks/LockNotReleasedException.java        |   24 -
 .../org/jgroups/blocks/LockingException.java    |   38 -
 .../org/jgroups/blocks/LogicalLink.java         |  340 --
 .../blocks/MembershipListenerAdapter.java       |   91 -
 .../org/jgroups/blocks/MessageDispatcher.java   |  845 ----
 .../jgroups/blocks/MessageListenerAdapter.java  |  123 -
 .../gemstone/org/jgroups/blocks/MethodCall.java |  529 --
 .../org/jgroups/blocks/MethodLookup.java        |   15 -
 .../org/jgroups/blocks/NBMessageForm_NIO.java   |   94 -
 .../org/jgroups/blocks/NotificationBus.java     |  458 --
 .../org/jgroups/blocks/PullPushAdapter.java     |  434 --
 .../org/jgroups/blocks/ReplicatedHashtable.java |  530 --
 .../org/jgroups/blocks/ReplicatedTree.java      | 1121 -----
 .../org/jgroups/blocks/ReplicationData.java     |  148 -
 .../org/jgroups/blocks/ReplicationManager.java  |  345 --
 .../org/jgroups/blocks/ReplicationReceiver.java |   69 -
 .../org/jgroups/blocks/RequestCorrelator.java   |  911 ----
 .../org/jgroups/blocks/RequestHandler.java      |   15 -
 .../org/jgroups/blocks/RpcDispatcher.java       |  392 --
 .../org/jgroups/blocks/RspCollector.java        |   18 -
 .../jgroups/blocks/TwoPhaseVotingAdapter.java   |  173 -
 .../jgroups/blocks/TwoPhaseVotingListener.java  |   35 -
 .../org/jgroups/blocks/UpdateException.java     |   19 -
 .../org/jgroups/blocks/VoteException.java       |   19 -
 .../jgroups/blocks/VoteResponseProcessor.java   |   31 -
 .../org/jgroups/blocks/VotingAdapter.java       |  472 --
 .../org/jgroups/blocks/VotingListener.java      |   26 -
 .../com/gemstone/org/jgroups/blocks/Xid.java    |  164 -
 .../gemstone/org/jgroups/blocks/package.html    |   13 -
 .../org/jgroups/conf/ClassConfigurator.java     |  228 -
 .../com/gemstone/org/jgroups/conf/ClassMap.java |   75 -
 .../jgroups/conf/ClassPathEntityResolver.java   |   63 -
 .../org/jgroups/conf/ConfiguratorFactory.java   |  449 --
 .../org/jgroups/conf/MagicNumberReader.java     |  402 --
 .../org/jgroups/conf/PlainConfigurator.java     |   48 -
 .../gemstone/org/jgroups/conf/ProtocolData.java |  130 -
 .../org/jgroups/conf/ProtocolParameter.java     |   64 -
 .../jgroups/conf/ProtocolStackConfigurator.java |   18 -
 .../org/jgroups/conf/XmlConfigurator.java       |  463 --
 .../gemstone/org/jgroups/conf/XmlValidator.java |  146 -
 .../com/gemstone/org/jgroups/conf/package.html  |    5 -
 .../gemstone/org/jgroups/debug/Debugger.java    |  133 -
 .../org/jgroups/debug/JChannelTestHook.java     |   14 -
 .../gemstone/org/jgroups/debug/Profiler.java    |  160 -
 .../org/jgroups/debug/ProtocolSpecificView.java |   24 -
 .../org/jgroups/debug/ProtocolTester.java       |  142 -
 .../org/jgroups/debug/ProtocolView.java         |   90 -
 .../gemstone/org/jgroups/debug/QUEUEView.java   |   28 -
 .../gemstone/org/jgroups/debug/Simulator.java   |  249 -
 .../com/gemstone/org/jgroups/debug/package.html |    5 -
 .../com/gemstone/org/jgroups/gemstonizing.txt   |   28 -
 .../org/jgroups/oswego/concurrent/Barrier.java  |   65 -
 .../oswego/concurrent/BoundedBuffer.java        |  190 -
 .../oswego/concurrent/BoundedChannel.java       |   37 -
 .../oswego/concurrent/BoundedLinkedQueue.java   |  384 --
 .../oswego/concurrent/BoundedPriorityQueue.java |  123 -
 .../concurrent/BrokenBarrierException.java      |   48 -
 .../org/jgroups/oswego/concurrent/Callable.java |   39 -
 .../org/jgroups/oswego/concurrent/Channel.java  |  309 --
 .../jgroups/oswego/concurrent/ClockDaemon.java  |  403 --
 .../org/jgroups/oswego/concurrent/CondVar.java  |  277 --
 .../jgroups/oswego/concurrent/CountDown.java    |  126 -
 .../oswego/concurrent/CyclicBarrier.java        |  299 --
 .../concurrent/DefaultChannelCapacity.java      |   58 -
 .../oswego/concurrent/DirectExecutor.java       |   36 -
 .../org/jgroups/oswego/concurrent/Executor.java |   70 -
 .../oswego/concurrent/FIFOReadWriteLock.java    |  198 -
 .../oswego/concurrent/FIFOSemaphore.java        |   84 -
 .../org/jgroups/oswego/concurrent/FJTask.java   |  535 --
 .../jgroups/oswego/concurrent/FJTaskRunner.java |  979 ----
 .../oswego/concurrent/FJTaskRunnerGroup.java    |  625 ---
 .../jgroups/oswego/concurrent/FutureResult.java |  209 -
 .../org/jgroups/oswego/concurrent/Heap.java     |  151 -
 .../org/jgroups/oswego/concurrent/Latch.java    |  103 -
 .../jgroups/oswego/concurrent/LayeredSync.java  |   96 -
 .../jgroups/oswego/concurrent/LinkedNode.java   |   29 -
 .../jgroups/oswego/concurrent/LinkedQueue.java  |  192 -
 .../oswego/concurrent/LockedExecutor.java       |   62 -
 .../org/jgroups/oswego/concurrent/Mutex.java    |  173 -
 .../org/jgroups/oswego/concurrent/NullSync.java |   51 -
 .../oswego/concurrent/PooledExecutor.java       |  924 ----
 .../oswego/concurrent/PrioritySemaphore.java    |   96 -
 .../concurrent/PropertyChangeMulticaster.java   |  466 --
 .../org/jgroups/oswego/concurrent/Puttable.java |   68 -
 .../oswego/concurrent/QueuedExecutor.java       |  220 -
 .../oswego/concurrent/QueuedSemaphore.java      |  181 -
 .../oswego/concurrent/ReadWriteLock.java        |   86 -
 .../ReaderPreferenceReadWriteLock.java          |   35 -
 .../oswego/concurrent/ReentrantLock.java        |  151 -
 .../ReentrantWriterPreferenceReadWriteLock.java |  169 -
 .../jgroups/oswego/concurrent/Rendezvous.java   |  422 --
 .../jgroups/oswego/concurrent/Semaphore.java    |  193 -
 .../concurrent/SemaphoreControlledChannel.java  |  164 -
 .../org/jgroups/oswego/concurrent/Slot.java     |   88 -
 .../org/jgroups/oswego/concurrent/Sync.java     |  344 --
 .../oswego/concurrent/SyncCollection.java       |  514 --
 .../org/jgroups/oswego/concurrent/SyncList.java |  327 --
 .../org/jgroups/oswego/concurrent/SyncMap.java  |  314 --
 .../org/jgroups/oswego/concurrent/SyncSet.java  |   82 -
 .../oswego/concurrent/SyncSortedMap.java        |  129 -
 .../oswego/concurrent/SyncSortedSet.java        |  129 -
 .../oswego/concurrent/SynchronizedBoolean.java  |  182 -
 .../oswego/concurrent/SynchronizedByte.java     |  253 -
 .../oswego/concurrent/SynchronizedChar.java     |  176 -
 .../oswego/concurrent/SynchronizedDouble.java   |  181 -
 .../oswego/concurrent/SynchronizedFloat.java    |  181 -
 .../oswego/concurrent/SynchronizedInt.java      |  250 -
 .../oswego/concurrent/SynchronizedLong.java     |  252 -
 .../oswego/concurrent/SynchronizedRef.java      |  107 -
 .../oswego/concurrent/SynchronizedShort.java    |  254 -
 .../oswego/concurrent/SynchronizedVariable.java |  209 -
 .../oswego/concurrent/SynchronousChannel.java   |  379 --
 .../org/jgroups/oswego/concurrent/Takable.java  |   67 -
 .../oswego/concurrent/ThreadFactory.java        |   35 -
 .../oswego/concurrent/ThreadFactoryUser.java    |   62 -
 .../oswego/concurrent/ThreadedExecutor.java     |   40 -
 .../oswego/concurrent/TimedCallable.java        |   68 -
 .../oswego/concurrent/TimeoutException.java     |   53 -
 .../jgroups/oswego/concurrent/TimeoutSync.java  |   75 -
 .../concurrent/VetoableChangeMulticaster.java   |  577 ---
 .../oswego/concurrent/WaitFreeQueue.java        |  208 -
 .../oswego/concurrent/WaitableBoolean.java      |  141 -
 .../jgroups/oswego/concurrent/WaitableByte.java |  238 -
 .../jgroups/oswego/concurrent/WaitableChar.java |  170 -
 .../oswego/concurrent/WaitableDouble.java       |  173 -
 .../oswego/concurrent/WaitableFloat.java        |  173 -
 .../jgroups/oswego/concurrent/WaitableInt.java  |  239 -
 .../jgroups/oswego/concurrent/WaitableLong.java |  239 -
 .../jgroups/oswego/concurrent/WaitableRef.java  |  112 -
 .../oswego/concurrent/WaitableShort.java        |  238 -
 .../concurrent/WaiterPreferenceSemaphore.java   |  155 -
 .../WriterPreferenceReadWriteLock.java          |  337 --
 .../org/jgroups/oswego/concurrent/package.html  |  998 ----
 .../java/com/gemstone/org/jgroups/overview.html |   15 -
 .../java/com/gemstone/org/jgroups/package.html  |    5 -
 .../persistence/CannotConnectException.java     |   55 -
 .../CannotCreateSchemaException.java            |   45 -
 .../persistence/CannotPersistException.java     |   44 -
 .../persistence/CannotRemoveException.java      |   45 -
 .../persistence/CannotRetrieveException.java    |   44 -
 .../persistence/DBPersistenceManager.java       |  877 ----
 .../persistence/FilePersistenceManager.java     |  173 -
 .../jgroups/persistence/PersistenceFactory.java |  207 -
 .../jgroups/persistence/PersistenceManager.java |   69 -
 .../org/jgroups/persistence/package.html        |    5 -
 .../com/gemstone/org/jgroups/primarychanges.txt |   80 -
 .../gemstone/org/jgroups/protocols/AUTH.java    |  438 --
 .../org/jgroups/protocols/AUTOCONF.java         |  249 -
 .../org/jgroups/protocols/AuthHeader.java       |  108 -
 .../gemstone/org/jgroups/protocols/CAUSAL.java  |  296 --
 .../org/jgroups/protocols/COMPRESS.java         |  188 -
 .../org/jgroups/protocols/DEADLOCK.java         |  250 -
 .../gemstone/org/jgroups/protocols/DELAY.java   |  116 -
 .../com/gemstone/org/jgroups/protocols/DESIGN   |  283 --
 .../gemstone/org/jgroups/protocols/DISCARD.java |  150 -
 .../gemstone/org/jgroups/protocols/DUMMY.java   |   85 -
 .../org/jgroups/protocols/DUMMY_TP.java         |   84 -
 .../gemstone/org/jgroups/protocols/Digest.java  |   38 -
 .../org/jgroups/protocols/Discovery.java        |  484 --
 .../gemstone/org/jgroups/protocols/ENCRYPT.java | 1390 ------
 .../org/jgroups/protocols/ENCRYPT1_4.java       |  566 ---
 .../gemstone/org/jgroups/protocols/EXAMPLE.java |  104 -
 .../com/gemstone/org/jgroups/protocols/FC.java  | 1025 ----
 .../com/gemstone/org/jgroups/protocols/FD.java  | 1100 -----
 .../gemstone/org/jgroups/protocols/FD_PID.java  |  640 ---
 .../gemstone/org/jgroups/protocols/FD_PROB.java |  628 ---
 .../org/jgroups/protocols/FD_SIMPLE.java        |  357 --
 .../gemstone/org/jgroups/protocols/FD_SOCK.java | 2654 ----------
 .../org/jgroups/protocols/FD_SOCK.java.new      | 1153 -----
 .../org/jgroups/protocols/FLOWCONTROL.java      |   89 -
 .../org/jgroups/protocols/FLOW_CONTROL.java     |  317 --
 .../gemstone/org/jgroups/protocols/FLUSH.java   |  466 --
 .../gemstone/org/jgroups/protocols/FRAG.java    |  572 ---
 .../gemstone/org/jgroups/protocols/FRAG2.java   |  773 ---
 .../gemstone/org/jgroups/protocols/FRAG3.java   |   35 -
 .../org/jgroups/protocols/FlushRsp.java         |   31 -
 .../org/jgroups/protocols/FragHeader.java       |   80 -
 .../gemstone/org/jgroups/protocols/GMS.java.rmi |  219 -
 .../org/jgroups/protocols/GemFireTimeSync.java  |  720 ---
 .../gemstone/org/jgroups/protocols/HDRS.java    |   53 -
 .../gemstone/org/jgroups/protocols/HTOTAL.java  |  207 -
 .../org/jgroups/protocols/LOOPBACK.java         |  115 -
 .../gemstone/org/jgroups/protocols/LOSS.java    |  118 -
 .../gemstone/org/jgroups/protocols/MERGE.java   |  376 --
 .../gemstone/org/jgroups/protocols/MERGE2.java  |  362 --
 .../gemstone/org/jgroups/protocols/MERGE3.java  |  312 --
 .../org/jgroups/protocols/MERGEFAST.java        |  118 -
 .../protocols/MessageProtocolEXAMPLE.java       |   64 -
 .../protocols/NAKACK.java.MessageProtocol       |  422 --
 .../org/jgroups/protocols/NAKACK.java.separate  |  484 --
 .../org/jgroups/protocols/NakAckHeader.java     |  113 -
 .../org/jgroups/protocols/PARTITIONER.java      |  185 -
 .../gemstone/org/jgroups/protocols/PERF.java    |  284 --
 .../gemstone/org/jgroups/protocols/PERF_TP.java |  172 -
 .../org/jgroups/protocols/PIGGYBACK.java        |  271 --
 .../gemstone/org/jgroups/protocols/PING.java    |  297 --
 .../org/jgroups/protocols/PRINTMETHODS.java     |   64 -
 .../org/jgroups/protocols/PRINTOBJS.java        |  120 -
 .../org/jgroups/protocols/PerfHeader.java       |  451 --
 .../org/jgroups/protocols/PingHeader.java       |   79 -
 .../gemstone/org/jgroups/protocols/PingRsp.java |  107 -
 .../org/jgroups/protocols/PingSender.java       |  109 -
 .../org/jgroups/protocols/PingWaiter.java       |  301 --
 .../gemstone/org/jgroups/protocols/QUEUE.java   |  184 -
 .../jgroups/protocols/RpcProtocolEXAMPLE.java   |   64 -
 .../gemstone/org/jgroups/protocols/SHUFFLE.java |  146 -
 .../gemstone/org/jgroups/protocols/SIZE.java    |  186 -
 .../gemstone/org/jgroups/protocols/SMACK.java   |  393 --
 .../org/jgroups/protocols/STATE_TRANSFER.java   |  448 --
 .../gemstone/org/jgroups/protocols/STATS.java   |  211 -
 .../com/gemstone/org/jgroups/protocols/TCP.java |  308 --
 .../org/jgroups/protocols/TCPGOSSIP.java        |  429 --
 .../gemstone/org/jgroups/protocols/TCPPING.java |  144 -
 .../gemstone/org/jgroups/protocols/TCP_NIO.java |  118 -
 .../gemstone/org/jgroups/protocols/TOTAL.java   | 1055 ----
 .../com/gemstone/org/jgroups/protocols/TP.java  | 2012 --------
 .../gemstone/org/jgroups/protocols/TP.java.mt   | 1522 ------
 .../gemstone/org/jgroups/protocols/TRACE.java   |   47 -
 .../gemstone/org/jgroups/protocols/TUNNEL.java  |  459 --
 .../org/jgroups/protocols/TcpHeader.java        |   55 -
 .../org/jgroups/protocols/TpHeader.java         |   64 -
 .../protocols/TransportedVectorTime.java        |  185 -
 .../org/jgroups/protocols/TunnelHeader.java     |   47 -
 .../com/gemstone/org/jgroups/protocols/UDP.java | 1462 ------
 .../gemstone/org/jgroups/protocols/UDP.java.mt  | 1005 ----
 .../jgroups/protocols/UDP.java.packethandler    |  592 ---
 .../gemstone/org/jgroups/protocols/UDP_NIO.java | 1567 ------
 .../gemstone/org/jgroups/protocols/UNICAST.java |  977 ----
 .../org/jgroups/protocols/UdpHeader.java        |   62 -
 .../org/jgroups/protocols/VERIFY_SUSPECT.java   |  470 --
 .../org/jgroups/protocols/VIEW_ENFORCER.java    |   81 -
 .../org/jgroups/protocols/VIEW_SYNC.java        |  499 --
 .../org/jgroups/protocols/VectorTime.java       |  270 -
 .../gemstone/org/jgroups/protocols/WANPING.java |  106 -
 .../gemstone/org/jgroups/protocols/WANPIPE.java |  441 --
 .../org/jgroups/protocols/WanPipeAddress.java   |  140 -
 .../gemstone/org/jgroups/protocols/dump2.log    |    0
 .../jgroups/protocols/obsolete/ENCRYPT.java.txt |  408 --
 .../org/jgroups/protocols/obsolete/FC.java.txt  |  643 ---
 .../jgroups/protocols/obsolete/FD_RAND.java.txt |  287 --
 .../jgroups/protocols/obsolete/FD_SHUN.java.txt |  323 --
 .../org/jgroups/protocols/obsolete/TCP.java.txt |  493 --
 .../org/jgroups/protocols/obsolete/UDP.java.txt | 1921 --------
 .../jgroups/protocols/obsolete/UNIFORM.java.txt |  349 --
 .../gemstone/org/jgroups/protocols/package.html |    6 -
 .../jgroups/protocols/pbcast/ClientGmsImpl.java |  907 ----
 .../jgroups/protocols/pbcast/CoordGmsImpl.java  | 1103 -----
 .../org/jgroups/protocols/pbcast/DESIGN         |  478 --
 .../org/jgroups/protocols/pbcast/Digest.java    |  534 --
 .../jgroups/protocols/pbcast/Digest.java.old    |  558 ---
 .../org/jgroups/protocols/pbcast/FD.java        |  273 --
 .../org/jgroups/protocols/pbcast/GMS.java       | 2732 -----------
 .../org/jgroups/protocols/pbcast/GmsImpl.java   |  111 -
 .../org/jgroups/protocols/pbcast/Gossip.java    |  134 -
 .../org/jgroups/protocols/pbcast/JoinRsp.java   |  126 -
 .../org/jgroups/protocols/pbcast/MergeData.java |  118 -
 .../org/jgroups/protocols/pbcast/NAKACK.java    | 1631 -------
 .../jgroups/protocols/pbcast/NakAckHeader.java  |  148 -
 .../org/jgroups/protocols/pbcast/PBCAST.java    | 1043 ----
 .../protocols/pbcast/ParticipantGmsImpl.java    |  371 --
 .../jgroups/protocols/pbcast/PbcastHeader.java  |  110 -
 .../org/jgroups/protocols/pbcast/README         |  140 -
 .../org/jgroups/protocols/pbcast/STABLE.java    |  907 ----
 .../jgroups/protocols/pbcast/STABLE.java.latest |  897 ----
 .../jgroups/protocols/pbcast/STABLE.java.new    |  890 ----
 .../jgroups/protocols/pbcast/STABLE.java.old    |  855 ----
 .../protocols/pbcast/STATE_TRANSFER.java        |  461 --
 .../org/jgroups/protocols/pbcast/package.html   |    5 -
 .../org/jgroups/protocols/ring/RingNode.java    |   26 -
 .../protocols/ring/RingNodeFlowControl.java     |  135 -
 .../org/jgroups/protocols/ring/RingToken.java   |  245 -
 .../org/jgroups/protocols/ring/TcpRingNode.java |  205 -
 .../protocols/ring/TokenLostException.java      |   63 -
 .../org/jgroups/protocols/ring/package.html     |    5 -
 .../org/jgroups/spi/GFBasicAdapter.java         |   96 -
 .../gemstone/org/jgroups/spi/GFPeerAdapter.java |  169 -
 .../jgroups/stack/AckMcastReceiverWindow.java   |  169 -
 .../org/jgroups/stack/AckMcastSenderWindow.java |  601 ---
 .../org/jgroups/stack/AckReceiverWindow.java    |  184 -
 .../org/jgroups/stack/AckSenderWindow.java      |  354 --
 .../org/jgroups/stack/BoundedLinkedHashMap.java |   82 -
 .../org/jgroups/stack/Configurator.java         |  764 ---
 .../org/jgroups/stack/GFBasicAdapterImpl.java   |  580 ---
 .../org/jgroups/stack/GFPeerAdapterImpl.java    |  322 --
 .../org/jgroups/stack/GossipClient.java         |  819 ----
 .../gemstone/org/jgroups/stack/GossipData.java  |  232 -
 .../org/jgroups/stack/GossipServer.java         |  633 ---
 .../gemstone/org/jgroups/stack/Interval.java    |   40 -
 .../gemstone/org/jgroups/stack/IpAddress.java   |  780 ---
 .../org/jgroups/stack/LogicalAddress.java       |  370 --
 .../org/jgroups/stack/MessageProtocol.java      |  243 -
 .../org/jgroups/stack/NakReceiverWindow.java    |  914 ----
 .../gemstone/org/jgroups/stack/Protocol.java    |  837 ----
 .../org/jgroups/stack/ProtocolObserver.java     |   81 -
 .../org/jgroups/stack/ProtocolStack.java        |  506 --
 .../org/jgroups/stack/Retransmitter.java        |  480 --
 .../com/gemstone/org/jgroups/stack/Router.java  |  577 ---
 .../gemstone/org/jgroups/stack/RouterStub.java  |  419 --
 .../gemstone/org/jgroups/stack/RpcProtocol.java |  166 -
 .../org/jgroups/stack/SockCreatorImpl.java      |   48 -
 .../org/jgroups/stack/StateTransferInfo.java    |   64 -
 .../com/gemstone/org/jgroups/stack/package.html |    5 -
 .../gemstone/org/jgroups/util/AckCollector.java |  149 -
 .../gemstone/org/jgroups/util/BoundedList.java  |   53 -
 .../com/gemstone/org/jgroups/util/Buffer.java   |   64 -
 .../com/gemstone/org/jgroups/util/Command.java  |   16 -
 .../com/gemstone/org/jgroups/util/CondVar.java  |  139 -
 .../org/jgroups/util/ConnectionWatcher.java     |   25 -
 .../jgroups/util/ContextObjectInputStream.java  |   91 -
 .../util/ExposedBufferedInputStream.java        |   61 -
 .../util/ExposedBufferedOutputStream.java       |   44 -
 .../util/ExposedByteArrayInputStream.java       |   58 -
 .../util/ExposedByteArrayOutputStream.java      |   31 -
 .../jgroups/util/ExposedDataOutputStream.java   |   31 -
 .../org/jgroups/util/ExternalStrings.java       |  823 ----
 .../gemstone/org/jgroups/util/GFLogWriter.java  |   57 -
 .../org/jgroups/util/GFStringIdImpl.java        |   68 -
 .../org/jgroups/util/GemFireTracer.java         |  473 --
 .../org/jgroups/util/GetNetworkInterfaces.java  |   34 -
 .../org/jgroups/util/LinkedListQueue.java       |  422 --
 .../com/gemstone/org/jgroups/util/List.java     |  462 --
 .../jgroups/util/MagicObjectInputStream.java    |   55 -
 .../jgroups/util/MagicObjectOutputStream.java   |   60 -
 .../gemstone/org/jgroups/util/Marshaller.java   |  152 -
 .../org/jgroups/util/NullReadWriteLock.java     |   25 -
 .../com/gemstone/org/jgroups/util/NullSync.java |   26 -
 .../gemstone/org/jgroups/util/PrintXMLTree.java |  168 -
 .../com/gemstone/org/jgroups/util/Promise.java  |  162 -
 .../com/gemstone/org/jgroups/util/Proxy.java    |  870 ----
 .../com/gemstone/org/jgroups/util/Queue.java    |  653 ---
 .../org/jgroups/util/Queue.java.concurrent      |  113 -
 .../com/gemstone/org/jgroups/util/Queue2.java   |  715 ---
 .../org/jgroups/util/QueueClosedException.java  |   29 -
 .../com/gemstone/org/jgroups/util/Range.java    |   58 -
 .../org/jgroups/util/ReentrantLatch.java        |   65 -
 .../org/jgroups/util/ReusableThread.java        |  295 --
 .../java/com/gemstone/org/jgroups/util/Rsp.java |   82 -
 .../com/gemstone/org/jgroups/util/RspList.java  |  193 -
 .../gemstone/org/jgroups/util/Scheduler.java    |  270 -
 .../org/jgroups/util/SchedulerListener.java     |   29 -
 .../gemstone/org/jgroups/util/SockCreator.java  |   20 -
 .../com/gemstone/org/jgroups/util/Stack.java    |  108 -
 .../gemstone/org/jgroups/util/Streamable.java   |   26 -
 .../org/jgroups/util/StreamableFixedID.java     |   19 -
 .../com/gemstone/org/jgroups/util/StringId.java |   49 -
 .../gemstone/org/jgroups/util/ThreadPool.java   |  105 -
 .../org/jgroups/util/TimeScheduler.java         |  720 ---
 .../gemstone/org/jgroups/util/TimedWriter.java  |  295 --
 .../com/gemstone/org/jgroups/util/Util.java     | 1869 -------
 .../org/jgroups/util/VersionedStreamable.java   |   18 -
 .../com/gemstone/org/jgroups/util/package.html  |    5 -
 .../java/com/gemstone/org/jgroups/util/todo.txt |   80 -
 .../gemstone/org/jgroups/conf/jg-magic-map.dtd  |   55 -
 .../gemstone/org/jgroups/conf/jg-magic-map.xml  |  269 -
 .../com/gemstone/org/jgroups/log4j2-default.xml |   21 -
 .../org/jgroups/stack/jboss-service.xml         |  146 -
 .../gemstone/org/jgroups/JChannelJUnitTest.java |  271 --
 .../src/main/java/org/json/JSONObject.java      |    2 +
 .../test/junit/categories/DistributedTest.java  |   16 +
 .../categories/DistributedTransactionsTest.java |   16 +
 .../test/junit/categories/HoplogTest.java       |    7 -
 .../test/junit/categories/IntegrationTest.java  |   16 +
 .../test/junit/categories/PerformanceTest.java  |   16 +
 .../gemfire/test/junit/categories/UnitTest.java |   16 +
 .../gemfire/test/junit/categories/WanTest.java  |   16 +
 .../test/junit/rules/ExpectedTimeout.java       |   16 +
 .../junit/rules/ExpectedTimeoutJUnitTest.java   |   16 +
 gemfire-lucene/build.gradle                     |   46 +
 .../gemfire/cache/lucene/LuceneIndex.java       |   60 +
 .../gemfire/cache/lucene/LuceneQuery.java       |   48 +
 .../cache/lucene/LuceneQueryFactory.java        |  101 +
 .../cache/lucene/LuceneQueryProvider.java       |   45 +
 .../cache/lucene/LuceneQueryResults.java        |   58 +
 .../cache/lucene/LuceneResultStruct.java        |   62 +
 .../gemfire/cache/lucene/LuceneService.java     |  125 +
 .../cache/lucene/LuceneServiceProvider.java     |   46 +
 .../lucene/internal/InternalLuceneIndex.java    |   29 +
 .../lucene/internal/InternalLuceneService.java  |   29 +
 .../lucene/internal/LuceneEventListener.java    |   99 +
 .../LuceneIndexForPartitionedRegion.java        |  136 +
 .../LuceneIndexForReplicatedRegion.java         |   48 +
 .../cache/lucene/internal/LuceneIndexImpl.java  |  107 +
 .../lucene/internal/LuceneQueryFactoryImpl.java |   67 +
 .../cache/lucene/internal/LuceneQueryImpl.java  |   87 +
 .../lucene/internal/LuceneQueryResultsImpl.java |  120 +
 .../lucene/internal/LuceneResultStructImpl.java |   94 +
 .../lucene/internal/LuceneServiceImpl.java      |  273 ++
 .../internal/PartitionedRepositoryManager.java  |  163 +
 .../lucene/internal/StringQueryProvider.java    |  106 +
 .../internal/directory/FileIndexInput.java      |  131 +
 .../internal/directory/RegionDirectory.java     |  119 +
 .../internal/distributed/CollectorManager.java  |   55 +
 .../lucene/internal/distributed/EntryScore.java |   82 +
 .../internal/distributed/LuceneFunction.java    |  137 +
 .../distributed/LuceneFunctionContext.java      |  115 +
 .../lucene/internal/distributed/TopEntries.java |  133 +
 .../distributed/TopEntriesCollector.java        |  102 +
 .../distributed/TopEntriesCollectorManager.java |  178 +
 .../TopEntriesFunctionCollector.java            |  163 +
 .../lucene/internal/filesystem/ChunkKey.java    |  123 +
 .../cache/lucene/internal/filesystem/File.java  |  155 +
 .../internal/filesystem/FileInputStream.java    |  166 +
 .../internal/filesystem/FileOutputStream.java   |  103 +
 .../lucene/internal/filesystem/FileSystem.java  |  156 +
 .../filesystem/SeekableInputStream.java         |   43 +
 .../internal/repository/IndexRepository.java    |   74 +
 .../repository/IndexRepositoryImpl.java         |  113 +
 .../repository/IndexResultCollector.java        |   47 +
 .../internal/repository/RepositoryManager.java  |   45 +
 .../HeterogenousLuceneSerializer.java           |   83 +
 .../repository/serializer/LuceneSerializer.java |   35 +
 .../serializer/PdxLuceneSerializer.java         |   47 +
 .../serializer/ReflectionLuceneSerializer.java  |   74 +
 .../repository/serializer/SerializerUtil.java   |  168 +
 .../internal/xml/LuceneIndexCreation.java       |  111 +
 .../internal/xml/LuceneIndexXmlGenerator.java   |   65 +
 .../internal/xml/LuceneServiceXmlGenerator.java |   39 +
 .../lucene/internal/xml/LuceneXmlConstants.java |   31 +
 .../lucene/internal/xml/LuceneXmlParser.java    |   97 +
 .../geode.apache.org/lucene/lucene-1.0.xsd      |   57 +
 ...gemstone.gemfire.internal.cache.CacheService |    1 +
 ...ne.gemfire.internal.cache.xmlcache.XmlParser |    1 +
 .../internal/LuceneEventListenerJUnitTest.java  |  109 +
 .../LuceneIndexRecoveryHAJUnitTest.java         |  201 +
 .../LuceneQueryFactoryImplJUnitTest.java        |   50 +
 .../internal/LuceneQueryImplJUnitTest.java      |  123 +
 .../LuceneQueryResultsImplJUnitTest.java        |  126 +
 .../LuceneResultStructImpJUnitTest.java         |   51 +
 .../internal/LuceneServiceImplJUnitTest.java    |  226 +
 .../PartitionedRepositoryManagerJUnitTest.java  |  230 +
 .../internal/StringQueryProviderJUnitTest.java  |   90 +
 .../directory/RegionDirectoryJUnitTest.java     |   56 +
 .../DistributedScoringJUnitTest.java            |  155 +
 .../distributed/EntryScoreJUnitTest.java        |   40 +
 .../LuceneFunctionContextJUnitTest.java         |   64 +
 .../distributed/LuceneFunctionJUnitTest.java    |  423 ++
 .../LuceneFunctionReadPathDUnitTest.java        |  241 +
 .../TopEntriesCollectorJUnitTest.java           |  139 +
 .../TopEntriesFunctionCollectorJUnitTest.java   |  323 ++
 .../distributed/TopEntriesJUnitTest.java        |  146 +
 .../internal/filesystem/ChunkKeyJUnitTest.java  |   48 +
 .../internal/filesystem/FileJUnitTest.java      |   53 +
 .../filesystem/FileSystemJUnitTest.java         |  578 +++
 ...IndexRepositoryImplJUnitPerformanceTest.java |  437 ++
 .../IndexRepositoryImplJUnitTest.java           |  208 +
 .../HeterogenousLuceneSerializerJUnitTest.java  |   90 +
 .../serializer/PdxFieldMapperJUnitTest.java     |   85 +
 .../ReflectionFieldMapperJUnitTest.java         |   85 +
 .../internal/repository/serializer/Type1.java   |   48 +
 .../internal/repository/serializer/Type2.java   |   34 +
 ...neIndexXmlGeneratorIntegrationJUnitTest.java |   78 +
 .../xml/LuceneIndexXmlGeneratorJUnitTest.java   |   80 +
 ...uceneIndexXmlParserIntegrationJUnitTest.java |  107 +
 .../xml/LuceneIndexXmlParserJUnitTest.java      |   72 +
 ...erIntegrationJUnitTest.createIndex.cache.xml |   41 +
 ...serIntegrationJUnitTest.parseIndex.cache.xml |   41 +
 gemfire-rebalancer/build.gradle                 |   30 +
 .../gemfire/cache/util/AutoBalancer.java        |  554 +++
 .../util/AutoBalancerIntegrationJUnitTest.java  |  206 +
 .../cache/util/AutoBalancerJUnitTest.java       |  604 +++
 gemfire-site/.gitignore                         |    1 +
 gemfire-site/build.gradle                       |   37 -
 gemfire-site/src/jbake.zip                      |  Bin 207030 -> 0 bytes
 gemfire-site/src/jbake/assets/favicon.ico       |  Bin 1150 -> 0 bytes
 .../src/jbake/assets/images/bg-billboard.png    |  Bin 25538 -> 0 bytes
 .../jbake/assets/images/bg-crystals-home.png    |  Bin 41684 -> 0 bytes
 .../assets/images/bg-crystals-secondary.png     |  Bin 26046 -> 0 bytes
 .../src/jbake/assets/images/egg-logo1.png       |  Bin 8626 -> 0 bytes
 .../jbake/assets/images/events/apachecon.png    |  Bin 4528 -> 0 bytes
 .../src/jbake/assets/images/events/oscon.png    |  Bin 26024 -> 0 bytes
 .../src/jbake/assets/images/geode-banner.png    |  Bin 7916 -> 0 bytes
 .../assets/images/logo-apache-geode-white.png   |  Bin 2336 -> 0 bytes
 .../jbake/assets/images/logo-apache-geode.png   |  Bin 3200 -> 0 bytes
 .../jbake/assets/images/logo-geode-white.png    |  Bin 1620 -> 0 bytes
 .../src/jbake/assets/images/logo-geode.png      |  Bin 3345 -> 0 bytes
 .../src/jbake/assets/javascripts/master.js      |  121 -
 .../src/jbake/assets/javascripts/scale.fix.js   |   20 -
 .../jbake/assets/stylesheets/pygment_trac.css   |   60 -
 .../src/jbake/assets/stylesheets/styles.css     |  319 --
 gemfire-site/src/jbake/content/404.md           |    9 -
 gemfire-site/src/jbake/content/README.md        |   36 -
 gemfire-site/src/jbake/content/about/index.md   |   31 -
 .../src/jbake/content/community/index.md        |   82 -
 .../src/jbake/content/contribute/index.md       |   47 -
 gemfire-site/src/jbake/content/docs/index.md    |   23 -
 .../src/jbake/content/download/index.md         |   13 -
 .../src/jbake/content/getting-started/index.md  |   88 -
 gemfire-site/src/jbake/content/index.md         |   76 -
 gemfire-site/src/jbake/jbake.properties         |    6 -
 gemfire-site/src/jbake/templates/page.groovy    |   80 -
 gemfire-site/website/.gitignore                 |    1 +
 gemfire-site/website/README.md                  |   54 +
 gemfire-site/website/Rules                      |   69 +
 gemfire-site/website/build.sh                   |   18 +
 .../website/content/bootstrap/bootstrap.min.css |    9 +
 .../website/content/community/index.html        |  302 ++
 .../website/content/css/bootflat-extensions.css |  356 ++
 .../website/content/css/bootflat-square.css     |   69 +
 gemfire-site/website/content/css/bootflat.css   | 1559 ++++++
 .../website/content/css/font-awesome.min.css    |  405 ++
 gemfire-site/website/content/css/geode-site.css | 1570 ++++++
 gemfire-site/website/content/favicon.ico        |  Bin 0 -> 20805 bytes
 .../website/content/font/FontAwesome.otf        |  Bin 0 -> 61896 bytes
 .../content/font/fontawesome-webfont-eot.eot    |  Bin 0 -> 37405 bytes
 .../content/font/fontawesome-webfont-svg.svg    |  399 ++
 .../content/font/fontawesome-webfont-ttf.ttf    |  Bin 0 -> 79076 bytes
 .../content/font/fontawesome-webfont-woff.woff  |  Bin 0 -> 43572 bytes
 .../website/content/img/apache_geode_logo.png   |  Bin 0 -> 23616 bytes
 .../content/img/apache_geode_logo_white.png     |  Bin 0 -> 22695 bytes
 .../img/apache_geode_logo_white_small.png       |  Bin 0 -> 52948 bytes
 .../website/content/img/check_flat/default.png  |  Bin 0 -> 25851 bytes
 gemfire-site/website/content/img/egg-logo.png   |  Bin 0 -> 9938 bytes
 gemfire-site/website/content/img/github.png     |  Bin 0 -> 8936 bytes
 gemfire-site/website/content/index.html         |  140 +
 .../website/content/js/bootstrap.min.js         |    8 +
 gemfire-site/website/content/js/head.js         |  708 +++
 gemfire-site/website/content/js/html5shiv.js    |    8 +
 .../website/content/js/jquery-1.10.1.min.js     |    6 +
 .../website/content/js/jquery.icheck.js         |  397 ++
 gemfire-site/website/content/js/respond.min.js  |    6 +
 .../website/content/js/usergrid-site.js         |   66 +
 .../website/content/releases/index.html         |   82 +
 gemfire-site/website/layouts/community.html     |    1 +
 gemfire-site/website/layouts/default.html       |   44 +
 gemfire-site/website/layouts/docs.html          |    1 +
 gemfire-site/website/layouts/footer.html        |   96 +
 gemfire-site/website/layouts/header.html        |  247 +
 gemfire-site/website/lib/default.rb             |   60 +
 gemfire-site/website/lib/helpers_.rb            |   16 +
 gemfire-site/website/lib/pandoc.template        |    4 +
 gemfire-site/website/nanoc.yaml                 |   94 +
 gemfire-site/website/run.sh                     |   18 +
 gemfire-site/website/utilities/map-markers.rb   |   75 +
 gemfire-site/website/utilities/markers.txt      |  440 ++
 .../website/utilities/snapshot-apigee.rb        |   88 +
 gemfire-spark-connector/.gitignore              |    1 +
 gemfire-spark-connector/README.md               |   32 +
 gemfire-spark-connector/doc/10_demos.md         |   84 +
 gemfire-spark-connector/doc/1_building.md       |   36 +
 gemfire-spark-connector/doc/2_quick.md          |  178 +
 gemfire-spark-connector/doc/3_connecting.md     |   55 +
 gemfire-spark-connector/doc/4_loading.md        |  108 +
 gemfire-spark-connector/doc/5_rdd_join.md       |  237 +
 gemfire-spark-connector/doc/6_save_rdd.md       |   81 +
 gemfire-spark-connector/doc/7_save_dstream.md   |   68 +
 gemfire-spark-connector/doc/8_oql.md            |   58 +
 gemfire-spark-connector/doc/9_java_api.md       |  129 +
 .../connector/internal/RegionMetadata.java      |   93 +
 .../gemfirefunctions/QueryFunction.java         |   99 +
 .../RetrieveRegionFunction.java                 |  208 +
 .../RetrieveRegionMetadataFunction.java         |  118 +
 .../StructStreamingResultSender.java            |  219 +
 .../gemfire/spark/connector/Employee.java       |   54 +
 .../spark/connector/JavaApiIntegrationTest.java |  424 ++
 .../gemfire/spark/connector/Portfolio.java      |  109 +
 .../gemfire/spark/connector/Position.java       |   73 +
 .../src/it/resources/test-regions.xml           |   49 +
 .../src/it/resources/test-retrieve-regions.xml  |   57 +
 .../spark/connector/BasicIntegrationTest.scala  |  598 +++
 .../RDDJoinRegionIntegrationTest.scala          |  300 ++
 .../RetrieveRegionIntegrationTest.scala         |  253 +
 .../gemfire/spark/connector/package.scala       |   29 +
 .../connector/testkit/GemFireCluster.scala      |   47 +
 .../spark/connector/testkit/GemFireRunner.scala |  148 +
 .../spark/connector/testkit/IOUtils.scala       |   94 +
 .../spark/streaming/ManualClockHelper.scala     |   28 +
 .../spark/streaming/TestInputDStream.scala      |   44 +
 .../javaapi/GemFireJavaDStreamFunctions.java    |   86 +
 .../GemFireJavaPairDStreamFunctions.java        |   77 +
 .../javaapi/GemFireJavaPairRDDFunctions.java    |  238 +
 .../javaapi/GemFireJavaRDDFunctions.java        |  178 +
 .../javaapi/GemFireJavaSQLContextFunctions.java |   49 +
 .../GemFireJavaSparkContextFunctions.java       |   87 +
 .../connector/javaapi/GemFireJavaUtil.java      |  122 +
 .../spark/connector/GemFireConnection.scala     |   67 +
 .../spark/connector/GemFireConnectionConf.scala |   73 +
 .../connector/GemFireConnectionManager.scala    |   31 +
 .../connector/GemFireFunctionDeployer.scala     |   81 +
 .../connector/GemFireKryoRegistrator.scala      |   29 +
 .../connector/GemFirePairRDDFunctions.scala     |  140 +
 .../spark/connector/GemFireRDDFunctions.scala   |  120 +
 .../connector/GemFireSQLContextFunctions.scala  |   42 +
 .../GemFireSparkContextFunctions.scala          |   39 +
 .../internal/DefaultGemFireConnection.scala     |  164 +
 .../DefaultGemFireConnectionManager.scala       |   77 +
 .../connector/internal/LocatorHelper.scala      |  135 +
 .../StructStreamingResultCollector.scala        |  152 +
 .../connector/internal/oql/QueryParser.scala    |   58 +
 .../spark/connector/internal/oql/QueryRDD.scala |   83 +
 .../internal/oql/QueryResultCollector.scala     |   69 +
 .../connector/internal/oql/RDDConverter.scala   |   40 +
 .../connector/internal/oql/RowBuilder.scala     |   38 +
 .../connector/internal/oql/SchemaBuilder.scala  |   73 +
 .../internal/oql/UndefinedSerializer.scala      |   46 +
 .../connector/internal/rdd/GemFireJoinRDD.scala |   67 +
 .../internal/rdd/GemFireOuterJoinRDD.scala      |   69 +
 .../internal/rdd/GemFireRDDPartition.scala      |   36 +
 .../internal/rdd/GemFireRDDPartitioner.scala    |   59 +
 .../rdd/GemFireRDDPartitionerImpl.scala         |   89 +
 .../internal/rdd/GemFireRDDWriter.scala         |   82 +
 .../internal/rdd/GemFireRegionRDD.scala         |  138 +
 .../javaapi/GemFireJavaRegionRDD.scala          |   26 +
 .../spark/connector/javaapi/JavaAPIHelper.scala |   53 +
 .../gemfire/spark/connector/package.scala       |   69 +
 .../streaming/GemFireDStreamFunctions.scala     |   89 +
 .../spark/connector/streaming/package.scala     |   32 +
 .../gemfire/spark/connector/JavaAPITest.java    |  163 +
 .../connector/GemFireFunctionDeployerTest.scala |   58 +
 .../DefaultGemFireConnectionManagerTest.scala   |   82 +
 ...tStreamingResultSenderAndCollectorTest.scala |  254 +
 .../internal/oql/QueryParserTest.scala          |   83 +
 .../connector/ConnectorImplicitsTest.scala      |   50 +
 .../connector/GemFireConnectionConfTest.scala   |  100 +
 .../connector/GemFireDStreamFunctionsTest.scala |   79 +
 .../connector/GemFireRDDFunctionsTest.scala     |  139 +
 .../spark/connector/LocatorHelperTest.scala     |  168 +
 .../rdd/GemFireRDDPartitionerTest.scala         |  190 +
 .../connector/rdd/GemFireRegionRDDTest.scala    |  117 +
 .../basic-demos/src/main/java/demo/Emp.java     |   95 +
 .../src/main/java/demo/OQLJavaDemo.java         |   59 +
 .../src/main/java/demo/PairRDDSaveJavaDemo.java |   86 +
 .../src/main/java/demo/RDDSaveJavaDemo.java     |   85 +
 .../src/main/java/demo/RegionToRDDJavaDemo.java |   57 +
 .../src/main/scala/demo/NetworkWordCount.scala  |   75 +
 .../project/Dependencies.scala                  |   45 +
 .../project/GemFireSparkBuild.scala             |   76 +
 gemfire-spark-connector/project/Settings.scala  |   57 +
 .../project/build.properties                    |    1 +
 gemfire-spark-connector/project/plugins.sbt     |    8 +
 gemfire-spark-connector/scalastyle-config.xml   |  117 +
 gemfire-web-api/build.gradle                    |   63 +-
 .../web/controllers/AbstractBaseController.java |   21 +-
 .../web/controllers/BaseControllerAdvice.java   |   21 +-
 .../web/controllers/CommonCrudController.java   |   21 +-
 .../controllers/FunctionAccessController.java   |   21 +-
 .../web/controllers/PdxBasedCrudController.java |   21 +-
 .../web/controllers/QueryAccessController.java  |   21 +-
 .../web/controllers/support/JSONTypes.java      |   16 +
 .../controllers/support/QueryResultTypes.java   |   21 +-
 .../web/controllers/support/RegionData.java     |   21 +-
 .../controllers/support/RegionEntryData.java    |   21 +-
 .../support/RestServersResultCollector.java     |   16 +
 .../web/controllers/support/UpdateOp.java       |   21 +-
 .../DataTypeNotSupportedException.java          |   21 +-
 .../web/exception/GemfireRestException.java     |   21 +-
 .../web/exception/MalformedJsonException.java   |   21 +-
 .../web/exception/RegionNotFoundException.java  |   21 +-
 .../exception/ResourceNotFoundException.java    |   21 +-
 ...stomMappingJackson2HttpMessageConverter.java |   16 +
 .../web/swagger/config/RestApiPathProvider.java |   29 +-
 .../web/swagger/config/SwaggerConfig.java       |   16 +
 .../rest/internal/web/util/ArrayUtils.java      |   21 +-
 .../rest/internal/web/util/DateTimeUtils.java   |   21 +-
 .../internal/web/util/IdentifiableUtils.java    |   21 +-
 .../rest/internal/web/util/JSONUtils.java       |   21 +-
 .../rest/internal/web/util/JsonWriter.java      |   21 +-
 .../rest/internal/web/util/NumberUtils.java     |   21 +-
 .../rest/internal/web/util/ValidationUtils.java |   21 +-
 .../main/webapp/WEB-INF/gemfire-api-servlet.xml |   16 +
 gemfire-web-api/src/main/webapp/WEB-INF/web.xml |   16 +
 gemfire-web/build.gradle                        |   49 +-
 .../src/main/webapp/WEB-INF/gemfire-servlet.xml |   16 +
 gemfire-web/src/main/webapp/WEB-INF/web.xml     |   16 +
 .../internal/web/AbstractWebTestCase.java       |   21 +-
 .../ShellCommandsControllerJUnitTest.java       |   68 +-
 ...entVariablesHandlerInterceptorJUnitTest.java |   21 +-
 .../internal/web/domain/LinkIndexJUnitTest.java |   21 +-
 .../internal/web/domain/LinkJUnitTest.java      |   21 +-
 .../domain/QueryParameterSourceJUnitTest.java   |   21 +-
 .../web/http/ClientHttpRequestJUnitTest.java    |   21 +-
 ...ableObjectHttpMessageConverterJUnitTest.java |   21 +-
 .../RestHttpOperationInvokerJUnitTest.java      |   21 +-
 .../SimpleHttpOperationInvokerJUnitTest.java    |   21 +-
 .../web/util/ConvertUtilsJUnitTest.java         |   21 +-
 .../internal/web/util/UriUtilsJUnitTest.java    |   21 +-
 gradle.properties                               |   14 +
 gradle/dependency-versions.properties           |   82 +
 gradle/rat.gradle                               |  163 +
 gradle/wrapper/gradle-wrapper.jar               |  Bin 51018 -> 53637 bytes
 gradle/wrapper/gradle-wrapper.properties        |    4 +-
 gradlew                                         |   12 +-
 settings.gradle                                 |   20 +-
 6049 files changed, 187709 insertions(+), 248996 deletions(-)
----------------------------------------------------------------------



[14/50] [abbrv] incubator-geode git commit: Ignoring tags file from ctags in rat

Posted by je...@apache.org.
Ignoring tags file from ctags in rat

My build was hanging on the rat step. After some investigation, I found
rat was stuck trying to process the tags file for some reason.


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

Branch: refs/heads/feature/GEODE-14
Commit: 4e4dd7c729c8047cf59c5dbdcdd8f96e4d71354f
Parents: 507f2f3
Author: Dan Smith <up...@apache.org>
Authored: Fri Dec 11 17:49:43 2015 -0800
Committer: Dan Smith <up...@apache.org>
Committed: Fri Dec 11 17:49:43 2015 -0800

----------------------------------------------------------------------
 gradle/rat.gradle | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4e4dd7c7/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index c0f40ec..f2026e6 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -38,6 +38,7 @@ rat {
     '**/build-eclipse/**',
     '*.iml',
     '.idea/**',
+    '**/tags',
 
     // text files
     '**/*.fig',


[05/50] [abbrv] incubator-geode git commit: GEODE-528: Locator not stopping correctly if jmx-manager-port=0

Posted by je...@apache.org.
GEODE-528: Locator not stopping correctly if jmx-manager-port=0


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

Branch: refs/heads/feature/GEODE-14
Commit: eb685b45c26260abb844dc8f92dbc2f50f728bb5
Parents: 80c61f4
Author: Jens Deppe <jd...@pivotal.io>
Authored: Thu Dec 10 13:15:45 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri Dec 11 13:59:15 2015 -0800

----------------------------------------------------------------------
 .../LocatorLauncherAssemblyJUnitTest.java       | 157 +++++++++++++++++++
 .../internal/SystemManagementService.java       |   3 +-
 2 files changed, 158 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb685b45/gemfire-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyJUnitTest.java b/gemfire-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyJUnitTest.java
new file mode 100644
index 0000000..1deff22
--- /dev/null
+++ b/gemfire-assembly/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherAssemblyJUnitTest.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
+import com.gemstone.gemfire.distributed.LocatorLauncher.Builder;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.process.ProcessType;
+import com.gemstone.gemfire.internal.process.ProcessUtils;
+import com.gemstone.gemfire.management.ManagementService;
+import com.gemstone.gemfire.management.ManagerMXBean;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.File;
+
+import static org.junit.Assert.*;
+
+/**
+ * These tests are part of assembly as they require the REST war file to be present.
+ *
+ * @author Jens Deppe
+ */
+@Category(IntegrationTest.class)
+public class LocatorLauncherAssemblyJUnitTest extends AbstractLocatorLauncherJUnitTestCase {
+
+  @Before
+  public final void setUpLocatorLauncherLocalTest() throws Exception {
+    disconnectFromDS();
+    System.setProperty(ProcessType.TEST_PREFIX_PROPERTY, getUniqueName() + "-");
+  }
+
+  @After
+  public final void tearDownLocatorLauncherLocalTest() throws Exception {
+    disconnectFromDS();
+  }
+
+  /*
+   * This test addresses GEODE-528
+   */
+  @Test
+  public void testLocatorStopsWhenJmxPortIsZero() throws Throwable {
+    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+
+    final Builder builder = new Builder()
+        .setMemberName(getUniqueName())
+        .setPort(this.locatorPort)
+        .setRedirectOutput(false)
+        .setWorkingDirectory(rootFolder)
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false")
+        .set(DistributionConfig.JMX_MANAGER_NAME, "true")
+        .set(DistributionConfig.JMX_MANAGER_START_NAME, "true")
+        .set(DistributionConfig.JMX_MANAGER_PORT_NAME, "0");
+
+    performTest(builder);
+  }
+
+  /*
+   * This test addresses GEODE-528
+   */
+  @Test
+  public void testLocatorStopsWhenJmxPortIsNonZero() throws Throwable {
+    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+    final int jmxPort = AvailablePortHelper.getRandomAvailableTCPPorts(1)[0];
+
+    final Builder builder = new Builder().setMemberName(getUniqueName())
+        .setPort(this.locatorPort)
+        .setRedirectOutput(false)
+        .setWorkingDirectory(rootFolder)
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false")
+        .set(DistributionConfig.JMX_MANAGER_NAME, "true")
+        .set(DistributionConfig.JMX_MANAGER_START_NAME, "true")
+        .set(DistributionConfig.JMX_MANAGER_PORT_NAME, Integer.toString(jmxPort));
+
+    performTest(builder);
+  }
+
+  private void performTest(Builder builder) {
+    assertFalse(builder.getForce());
+    this.launcher = builder.build();
+    assertFalse(this.launcher.isForcing());
+
+    LocatorLauncher dirLauncher = null;
+    int initialThreadCount = Thread.activeCount();
+
+    try {
+      this.launcher.start();
+      waitForLocatorToStart(this.launcher);
+
+      // validate the pid file and its contents
+      this.pidFile = new File(this.temporaryFolder.getRoot(), ProcessType.LOCATOR.getPidFileName());
+      assertTrue("Pid file " + this.pidFile.getCanonicalPath().toString() + " should exist", this.pidFile.exists());
+      final int pid = readPid(this.pidFile);
+      assertTrue(pid > 0);
+      assertEquals(ProcessUtils.identifyPid(), pid);
+
+      dirLauncher = new Builder().setWorkingDirectory(builder.getWorkingDirectory()).build();
+      assertNotNull(dirLauncher);
+      assertFalse(dirLauncher.isRunning());
+
+      // Stop the manager
+      Cache cache = CacheFactory.getAnyInstance();
+      ManagerMXBean managerBean = ManagementService.getManagementService(cache).getManagerMXBean();
+      managerBean.stop();
+
+      // stop the locator
+      final LocatorLauncher.LocatorState locatorState = dirLauncher.stop();
+      assertNotNull(locatorState);
+      assertEquals(Status.STOPPED, locatorState.getStatus());
+    } catch (Throwable e) {
+      this.errorCollector.addError(e);
+    }
+
+    try {
+      // verify the PID file was deleted
+      waitForFileToDelete(this.pidFile);
+    } catch (Throwable e) {
+      this.errorCollector.addError(e);
+    }
+
+    int finalThreadCount = Integer.MAX_VALUE;
+
+    // Spin for up to 5 seconds waiting for threads to finish
+    for (int i = 0; i < 50 && finalThreadCount > initialThreadCount; i++) {
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException ex) {
+        // ignored
+      }
+      finalThreadCount = Thread.activeCount();
+    }
+
+    assertEquals(initialThreadCount, finalThreadCount);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb685b45/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
index 68209f2..1ca10cc 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/SystemManagementService.java
@@ -527,12 +527,11 @@ public final class SystemManagementService extends BaseManagementService {
         federatingManager.stopManager();
         system.handleResourceEvent(ResourceEvent.MANAGER_STOP, null);
         getGemFireCacheImpl().getJmxManagerAdvisor().broadcastChange();
-        if (this.agent != null && this.agent.isRunning()) {
+        if (this.agent != null && (this.agent.isRunning() || this.agent.isHttpServiceRunning())) {
           this.agent.stopAgent();
         }
       }
     }
-
   }
 
   @Override


[06/50] [abbrv] incubator-geode git commit: StateFlushOperation flushing/waiting for multicast messages

Posted by je...@apache.org.
StateFlushOperation flushing/waiting for multicast messages

As part of the State Flush algorithm we record the state of the multicast
protocol in JGroups and send it to the initial image provider.  There the
information is used to wait for the on-wire multicast messages to be
received.

This change set also includes additional JGroupsMessenger tests for better code
coverage and fixes a few Find Bugs problems in that class and GMSJoinLeave.
One of these required replacing a volatile long with an AtomicLong because
the long was being incremented, which is not necessarily an atomic operation
on a volatile variable.


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

Branch: refs/heads/feature/GEODE-14
Commit: 2f0c7fcb5bd46ecc37af9341f43e8aa3521048e3
Parents: eb685b4
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Dec 11 15:06:46 2015 -0800
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Dec 11 15:09:08 2015 -0800

----------------------------------------------------------------------
 .../internal/direct/DirectChannel.java          |  2 +-
 .../internal/membership/MembershipManager.java  | 18 ++---
 .../membership/gms/interfaces/Messenger.java    | 22 +++++++
 .../membership/gms/membership/GMSJoinLeave.java | 10 +--
 .../gms/messenger/JGroupsMessenger.java         | 69 +++++++++++++++-----
 .../gms/mgr/GMSMembershipManager.java           | 20 ++----
 .../internal/cache/StateFlushOperation.java     |  6 +-
 .../gemfire/internal/tcp/ConnectionTable.java   |  2 +-
 .../gemfire/internal/tcp/TCPConduit.java        |  2 +-
 .../messenger/JGroupsMessengerJUnitTest.java    | 64 +++++++++++++++++-
 10 files changed, 165 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
index f84813e..14ff923 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/direct/DirectChannel.java
@@ -962,7 +962,7 @@ public class DirectChannel {
    *    the map to add the state to
    * @since 5.1
    */
-  public void getChannelStates(Stub member, HashMap result)
+  public void getChannelStates(Stub member, Map result)
   {
     TCPConduit tc = this.conduit;
     if (tc != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
index 54b82a7..a46680b 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/MembershipManager.java
@@ -177,30 +177,30 @@ public interface MembershipManager {
   public boolean shutdownInProgress();
 
   /**
-   * Returns a serializable map of communication channel state for
+   * Returns a serializable map of communications state for
    * use in state stabilization.
    * @param member
-   *    the member whose channel state is to be captured
+   *    the member whose message state is to be captured
    * @param includeMulticast
-   *    whether the state of the mcast channel should be included
+   *    whether the state of the mcast messaging should be included
    * @return the current state of the communication channels between this
    *    process and the given distributed member
    * @since 5.1
    */
-  public Map getChannelStates(DistributedMember member, boolean includeMulticast);
+  public Map getMessageState(DistributedMember member, boolean includeMulticast);
 
   /**
-   * Waits for the given communication channels to reach the associated
+   * Waits for the given communications to reach the associated
    * state
    * @param member
-   *    The member whose channel state we're waiting for
-   * @param channelState
-   *    The channel states to wait for.  This should come from getChannelStates
+   *    The member whose messaging state we're waiting for
+   * @param state
+   *    The message states to wait for.  This should come from getMessageStates
    * @throws InterruptedException
    *    Thrown if the thread is interrupted
    * @since 5.1
    */
-  public void waitForChannelState(DistributedMember member, Map channelState)
+  public void waitForMessageState(DistributedMember member, Map state)
     throws InterruptedException;
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Messenger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Messenger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Messenger.java
index 9def731..5bb6c4b 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Messenger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/interfaces/Messenger.java
@@ -16,6 +16,7 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.interfaces;
 
+import java.util.Map;
 import java.util.Set;
 
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
@@ -56,4 +57,25 @@ public interface Messenger extends Service {
    * @return true multicast is enabled and working
    */
   boolean testMulticast(long timeout) throws InterruptedException;
+
+  /**
+   * For the state-flush algorithm we need to be able to record
+   * the state of outgoing messages to the given member.  If multicast
+   * is being used for region operations we also need to record its
+   * state.
+   * 
+   * @param member the target member
+   * @param state messaging state is stored in this map
+   * @param includeMulticast whether to record multicast state
+   */
+  void getMessageState(InternalDistributedMember member, Map state, boolean includeMulticast);
+  
+  /**
+   * The flip-side of getMessageState, this method takes the state it recorded
+   * and waits for messages from the given member to be received.
+   * 
+   * @param member the member flushing operations to this member
+   * @param state the state of that member's outgoing messaging to this member
+   */
+  void waitForMessageState(InternalDistributedMember member, Map state) throws InterruptedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
index 3a3486b..abdceb4 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/membership/GMSJoinLeave.java
@@ -313,7 +313,8 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
    * @param coord
    * @return true if the attempt succeeded, false if it timed out
    */
-   boolean attemptToJoin() {
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="WA_NOT_IN_LOOP")
+  boolean attemptToJoin() {
     SearchState state = searchState;
 
     // send a join request to the coordinator and wait for a response
@@ -688,8 +689,6 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
 
   boolean sendView(NetView view, List<InternalDistributedMember> newMembers, boolean preparing, ViewReplyProcessor rp) {
 
-    boolean isNetworkPartition = isNetworkPartition(view, false);
-    
     int id = view.getViewId();
     InstallViewMessage msg = new InstallViewMessage(view, services.getAuthenticator().getCredentials(this.localAddress), preparing);
     Set<InternalDistributedMember> recips = new HashSet<>(view.getMembers());
@@ -954,6 +953,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
     }
   }    
 
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="WA_NOT_IN_LOOP")
   boolean findCoordinatorFromView() {
     ArrayList<FindCoordinatorResponse> result;
     SearchState state = searchState;
@@ -1799,7 +1799,9 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
         }
 
         // use the new view as the initial view
-        setInitialView(newView, newMembers, initialLeaving, initialRemovals);
+        synchronized(this) {
+          setInitialView(newView, newMembers, initialLeaving, initialRemovals);
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index 36a6200..bdf13b5 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -19,20 +19,16 @@ package com.gemstone.gemfire.distributed.internal.membership.gms.messenger;
 import static com.gemstone.gemfire.distributed.internal.membership.gms.GMSUtil.replaceStrings;
 import static com.gemstone.gemfire.internal.DataSerializableFixedID.JOIN_REQUEST;
 import static com.gemstone.gemfire.internal.DataSerializableFixedID.JOIN_RESPONSE;
-import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.io.OutputStream;
 import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -44,6 +40,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.logging.log4j.Logger;
 import org.jgroups.Address;
@@ -52,12 +49,12 @@ import org.jgroups.JChannel;
 import org.jgroups.Message;
 import org.jgroups.Message.Flag;
 import org.jgroups.Message.TransientFlag;
-import org.jgroups.Receiver;
 import org.jgroups.ReceiverAdapter;
 import org.jgroups.View;
 import org.jgroups.ViewId;
 import org.jgroups.conf.ClassConfigurator;
 import org.jgroups.protocols.UDP;
+import org.jgroups.protocols.pbcast.NAKACK2;
 import org.jgroups.stack.IpAddress;
 import org.jgroups.util.Digest;
 import org.jgroups.util.UUID;
@@ -97,6 +94,8 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.tcp.MemberShunnedException;
 
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+
 public class JGroupsMessenger implements Messenger {
 
   private static final Logger logger = Services.getLogger();
@@ -109,7 +108,7 @@ public class JGroupsMessenger implements Messenger {
   /**
    * The location (in the product) of the mcast Jgroups config file.
    */
-  private static final String DEFAULT_JGROUPS_MCAST_CONFIG = "com/gemstone/gemfire/distributed/internal/membership/gms/messenger/jgroups-mcast.xml";
+  private static final String JGROUPS_MCAST_CONFIG_FILE_NAME = "com/gemstone/gemfire/distributed/internal/membership/gms/messenger/jgroups-mcast.xml";
 
   /** JG magic numbers for types added to the JG ClassConfigurator */
   public static final short JGROUPS_TYPE_JGADDRESS = 2000;
@@ -127,13 +126,11 @@ public class JGroupsMessenger implements Messenger {
   /** handlers that receive certain classes of messages instead of the Manager */
   Map<Class, MessageHandler> handlers = new ConcurrentHashMap<Class, MessageHandler>();
   
-  Object nakackDigest;
-
   private volatile NetView view;
 
   private GMSPingPonger pingPonger = new GMSPingPonger();
   
-  protected volatile long pongsReceived;
+  protected AtomicLong pongsReceived = new AtomicLong(0);
   
   /**
    * A set that contains addresses that we have logged JGroups IOExceptions for in the
@@ -151,6 +148,7 @@ public class JGroupsMessenger implements Messenger {
   }
 
   @Override
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD")
   public void init(Services s) {
     this.services = s;
 
@@ -170,7 +168,7 @@ public class JGroupsMessenger implements Messenger {
 
     String r = null;
     if (transport.isMcastEnabled()) {
-      r = DEFAULT_JGROUPS_MCAST_CONFIG;
+      r = JGROUPS_MCAST_CONFIG_FILE_NAME;
     } else {
       r = DEFAULT_JGROUPS_TCP_CONFIG;
     }
@@ -248,6 +246,7 @@ public class JGroupsMessenger implements Messenger {
   }
 
   @Override
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD")
   public void start() {
     // create the configuration XML string for JGroups
     String properties = this.jgStackConfig;
@@ -303,8 +302,8 @@ public class JGroupsMessenger implements Messenger {
       throw new SystemConnectException("unable to create jgroups channel", e);
     }
     
-    if (THROW_EXCEPTION_ON_START_HOOK) {
-      THROW_EXCEPTION_ON_START_HOOK = false;
+    if (JGroupsMessenger.THROW_EXCEPTION_ON_START_HOOK) {
+      JGroupsMessenger.THROW_EXCEPTION_ON_START_HOOK = false;
       throw new SystemConnectException("failing for test");
     }
     
@@ -490,7 +489,7 @@ public class JGroupsMessenger implements Messenger {
   
   @Override
   public boolean testMulticast(long timeout) throws InterruptedException {
-    long pongsSnapshot = pongsReceived;
+    long pongsSnapshot = pongsReceived.longValue();
     JGAddress dest = null;
     try {
       pingPonger.sendPingMessage(myChannel, jgAddress, dest);
@@ -500,10 +499,48 @@ public class JGroupsMessenger implements Messenger {
       return false;
     }
     long giveupTime = System.currentTimeMillis() + timeout;
-    while (pongsReceived == pongsSnapshot && System.currentTimeMillis() < giveupTime) {
+    while (pongsReceived.longValue() == pongsSnapshot && System.currentTimeMillis() < giveupTime) {
       Thread.sleep(100);
     }
-    return pongsReceived > pongsSnapshot;
+    return pongsReceived.longValue() > pongsSnapshot;
+  }
+  
+  @Override
+  public void getMessageState(InternalDistributedMember target, Map state, boolean includeMulticast) {
+    if (includeMulticast) {
+      NAKACK2 nakack = (NAKACK2)myChannel.getProtocolStack().findProtocol("NAKACK2");
+      if (nakack != null) {
+        long seqno = nakack.getCurrentSeqno();
+        state.put("JGroups.mcastState", Long.valueOf(seqno));
+      }
+    }
+  }
+  
+  @Override
+  public void waitForMessageState(InternalDistributedMember sender, Map state) throws InterruptedException {
+    NAKACK2 nakack = (NAKACK2)myChannel.getProtocolStack().findProtocol("NAKACK2");
+    Long seqno = (Long)state.get("JGroups.mcastState");
+    if (nakack != null && seqno != null) {
+      waitForMessageState(nakack, sender, seqno);
+    }
+  }
+  
+  /**
+   * wait for the mcast state from the given member to reach the given seqno 
+   */
+  protected void waitForMessageState(NAKACK2 nakack, InternalDistributedMember sender, Long seqno)
+    throws InterruptedException {
+    JGAddress jgSender = new JGAddress(sender);
+    Digest digest = nakack.getDigest(jgSender);
+    if (digest != null) {
+      for (;;) {
+        long[] senderSeqnos = digest.get(jgSender);
+        if (senderSeqnos == null || senderSeqnos[0] >= seqno.longValue()) {
+          break;
+        }
+        Thread.sleep(50);
+      }
+    }
   }
 
   @Override
@@ -985,7 +1022,7 @@ public class JGroupsMessenger implements Messenger {
         }
         return;
       } else if (pingPonger.isPongMessage(contents)) {
-        pongsReceived++;
+        pongsReceived.incrementAndGet();
         return;
       }
       

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index e5835c4..0b7a544 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -39,7 +39,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.logging.log4j.Logger;
 
-import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.ForcedDisconnectException;
 import com.gemstone.gemfire.GemFireConfigException;
@@ -2366,17 +2365,18 @@ public class GMSMembershipManager implements MembershipManager, Manager
   /* non-thread-owned serial channels and high priority channels are not
    * included
    */
-  public Map getChannelStates(DistributedMember member, boolean includeMulticast) {
-    HashMap result = new HashMap();
+  public Map getMessageState(DistributedMember member, boolean includeMulticast) {
+    Map result = new HashMap();
     Stub stub = (Stub)memberToStubMap.get(member);
     DirectChannel dc = directChannel;
     if (stub != null && dc != null) {
       dc.getChannelStates(stub, result);
     }
+    services.getMessenger().getMessageState((InternalDistributedMember)member, result, includeMulticast);
     return result;
   }
 
-  public void waitForChannelState(DistributedMember otherMember, Map channelState)
+  public void waitForMessageState(DistributedMember otherMember, Map state)
     throws InterruptedException
   {
     if (Thread.interrupted()) throw new InterruptedException();
@@ -2389,15 +2389,9 @@ public class GMSMembershipManager implements MembershipManager, Manager
       latestViewLock.writeLock().unlock();
     }
     if (dc != null && stub != null) {
-      dc.waitForChannelState(stub, channelState);
-    }
-//    Long mcastState = (Long)channelState.remove("JGroups.MCast");
-//    if (mcastState != null) {
-//      InternalDistributedMember idm = (InternalDistributedMember)otherMember;
-//      GMSMember jgm = (GMSMember)idm.getNetMember();
-//      Address other = jgm.getAddress();
-//      gms.waitForMulticastState(other, mcastState.longValue());
-//    }
+      dc.waitForChannelState(stub, state);
+    }
+    services.getMessenger().waitForMessageState((InternalDistributedMember)otherMember, state);
   }
   
   /* 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/StateFlushOperation.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/StateFlushOperation.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/StateFlushOperation.java
index 8d3ea60..e56c126 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/StateFlushOperation.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/StateFlushOperation.java
@@ -131,7 +131,7 @@ public class StateFlushOperation  {
       gr.setRecipient(target);
       ReplyProcessor21 processor = new ReplyProcessor21(dm, target);
       gr.processorId = processor.getProcessorId();
-      gr.channelState = dm.getMembershipManager().getChannelStates(target, false);
+      gr.channelState = dm.getMembershipManager().getMessageState(target, false);
       if (logger.isTraceEnabled(LogMarker.STATE_FLUSH_OP) && ((gr.channelState != null) && (gr.channelState.size() > 0)) ) {
         logger.trace(LogMarker.STATE_FLUSH_OP, "channel states: {}", gr.channelStateDescription(gr.channelState));
       }
@@ -410,7 +410,7 @@ public class StateFlushOperation  {
               boolean useMulticast = r.getMulticastEnabled()
                                     && r.getSystem().getConfig().getMcastPort() != 0;
               if (initialized) {
-                Map channelStates = dm.getMembershipManager().getChannelStates(relayRecipient, useMulticast);
+                Map channelStates = dm.getMembershipManager().getMessageState(relayRecipient, useMulticast);
                 if (gr.channelState != null) {
                   gr.channelState.putAll(channelStates);
                 } else {
@@ -565,7 +565,7 @@ public class StateFlushOperation  {
                 dm.getCancelCriterion().checkCancelInProgress(null);
                 boolean interrupted = Thread.interrupted();
                 try {
-                  dm.getMembershipManager().waitForChannelState(getSender(), channelState);
+                  dm.getMembershipManager().waitForMessageState(getSender(), channelState);
                   break;
                 }
                 catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
index c866797..bac356c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/ConnectionTable.java
@@ -1080,7 +1080,7 @@ public class ConnectionTable  {
    * @since 5.1
    */
   protected void getThreadOwnedOrderedConnectionState(Stub member,
-      HashMap result) {
+      Map result) {
 
     ConcurrentMap cm = this.threadConnectionMap;
     if (cm != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
index 12a03fd..a2801c1 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
@@ -808,7 +808,7 @@ public class TCPConduit implements Runnable {
    */
   public void getThreadOwnedOrderedConnectionState(
     Stub member,
-    HashMap result)
+    Map result)
   {
     getConTable().getThreadOwnedOrderedConnectionState(member, result);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2f0c7fcb/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
index 4b9c01f..c844583 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
@@ -26,15 +26,20 @@ import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.commons.lang.SerializationException;
+import org.jgroups.Address;
 import org.jgroups.Event;
 import org.jgroups.JChannel;
 import org.jgroups.Message;
 import org.jgroups.conf.ClassConfigurator;
 import org.jgroups.protocols.UNICAST3;
+import org.jgroups.protocols.pbcast.NAKACK2;
+import org.jgroups.util.Digest;
 import org.jgroups.util.UUID;
 import org.junit.After;
 import org.junit.Test;
@@ -682,9 +687,9 @@ public class JGroupsMessengerJUnitTest {
 
     interceptor.collectedMessages.clear();
     JGroupsReceiver receiver = (JGroupsReceiver)messenger.myChannel.getReceiver();
-    long pongsReceived = messenger.pongsReceived;
+    long pongsReceived = messenger.pongsReceived.longValue();
     receiver.receive(pongMessage);
-    assertEquals(pongsReceived+1, messenger.pongsReceived);
+    assertEquals(pongsReceived+1, messenger.pongsReceived.longValue());
     receiver.receive(pingMessage);
     assertEquals("expected 1 message but found " + interceptor.collectedMessages, interceptor.collectedMessages.size(), 1);
     Message m = interceptor.collectedMessages.get(0);
@@ -756,6 +761,61 @@ public class JGroupsMessengerJUnitTest {
     assertTrue(newMessenger.myChannel == messenger.myChannel);
   }
   
+  @Test
+  public void testGetMessageState() throws Exception {
+    initMocks(true/*multicast*/);
+    messenger.testMulticast(50); // do some multicast messaging
+    NAKACK2 nakack = (NAKACK2)messenger.myChannel.getProtocolStack().findProtocol("NAKACK2");
+    assertNotNull(nakack);
+    long seqno = nakack.getCurrentSeqno();
+    Map state = new HashMap();
+    messenger.getMessageState(null, state, true);
+    assertEquals(1, state.size());
+    Long stateLong = (Long)state.values().iterator().next();
+    assertTrue("expected multicast state to be at least "+seqno+" but it was "+stateLong.longValue(),
+        stateLong.longValue() >= seqno);
+  }
+  
+  @Test
+  public void testGetMessageStateNoMulticast() throws Exception {
+    initMocks(false/*multicast*/);
+    Map state = new HashMap();
+    messenger.getMessageState(null, state, true);
+    assertEquals("expected an empty map but received " + state, 0, state.size());
+  }
+  
+  @Test
+  public void testWaitForMessageState() throws Exception {
+    initMocks(true/*multicast*/);
+    NAKACK2 nakack = mock(NAKACK2.class);
+    Digest digest = mock(Digest.class);
+    when(nakack.getDigest(any(Address.class))).thenReturn(digest);
+    when(digest.get(any(Address.class))).thenReturn(
+        new long[] {0,0}, new long[] {2, 50}, new long[] {49, 50}, new long[] {50, 80}, new long[] {80, 120});
+    messenger.waitForMessageState(nakack, createAddress(1234), Long.valueOf(50));
+    verify(digest, times(4)).get(isA(Address.class));
+    
+    reset(digest);
+    when(digest.get(any(Address.class))).thenReturn(
+        new long[] {0,0}, new long[] {2, 50}, null);
+    messenger.waitForMessageState(nakack, createAddress(1234), Long.valueOf(50));
+    verify(digest, times(3)).get(isA(Address.class));
+    
+    // for code coverage let's invoke the other waitForMessageState method
+    Map state = new HashMap();
+    state.put("JGroups.mcastState", Long.valueOf(10L));
+    messenger.waitForMessageState(createAddress(1234), state);
+  }
+  
+
+  @Test
+  public void testMulticastTest() throws Exception {
+    initMocks(true);
+    boolean result = messenger.testMulticast(50);
+    // this shouldln't succeed
+    assertFalse(result);
+  }
+  
   /**
    * creates an InternalDistributedMember address that can be used
    * with the doctored JGroups channel.  This includes a logical


[16/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/content/js/bootstrap.min.js
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/js/bootstrap.min.js b/gemfire-site/website/content/js/bootstrap.min.js
index 3137285..9c58496 100755
--- a/gemfire-site/website/content/js/bootstrap.min.js
+++ b/gemfire-site/website/content/js/bootstrap.min.js
@@ -5,4 +5,4 @@
 *
 * Designed and built with all the love in the world by @mdo and @fat.
 */
-if(!jQuery)throw new Error("Bootstrap requires jQuery");+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]}}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one(a.support.transition.end,function(){c=!0});var e=function(){c||a(d).trigger(a.support.transition.end)};return setTimeout(e,b),this},a(function(){a.support.transition=b()})}(window.jQuery),+function(a){"use strict";var b='[data-dismiss="alert"]',c=function(c){a(c).on("click",b,this.close)};c.prototype.close=function(b){function c(){f.trigger("closed.bs.alert").remove()}var d=a(this),e=d.attr("data-target");e||(e=d.attr("href"),e=e&&e.replace(/.*(?=#[^\s]*$)/,""));var f=a(e);b&&b.preventDefault(),f.length||(f=d.hasClass("alert")?d:d.parent()),f.trigger(b=a.Event("close.bs.alert")),b.isDefau
 ltPrevented()||(f.removeClass("in"),a.support.transition&&f.hasClass("fade")?f.one(a.support.transition.end,c).emulateTransitionEnd(150):c())};var d=a.fn.alert;a.fn.alert=function(b){return this.each(function(){var d=a(this),e=d.data("bs.alert");e||d.data("bs.alert",e=new c(this)),"string"==typeof b&&e[b].call(d)})},a.fn.alert.Constructor=c,a.fn.alert.noConflict=function(){return a.fn.alert=d,this},a(document).on("click.bs.alert.data-api",b,c.prototype.close)}(window.jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d)};b.DEFAULTS={loadingText:"loading..."},b.prototype.setState=function(a){var b="disabled",c=this.$element,d=c.is("input")?"val":"html",e=c.data();a+="Text",e.resetText||c.data("resetText",c[d]()),c[d](e[a]||this.options[a]),setTimeout(function(){"loadingText"==a?c.addClass(b).attr(b,b):c.removeClass(b).removeAttr(b)},0)},b.prototype.toggle=function(){var a=this.$element.closest('[data-toggle="buttons"]');if(a.l
 ength){var b=this.$element.find("input").prop("checked",!this.$element.hasClass("active")).trigger("change");"radio"===b.prop("type")&&a.find(".active").removeClass("active")}this.$element.toggleClass("active")};var c=a.fn.button;a.fn.button=function(c){return this.each(function(){var d=a(this),e=d.data("bs.button"),f="object"==typeof c&&c;e||d.data("bs.button",e=new b(this,f)),"toggle"==c?e.toggle():c&&e.setState(c)})},a.fn.button.Constructor=b,a.fn.button.noConflict=function(){return a.fn.button=c,this},a(document).on("click.bs.button.data-api","[data-toggle^=button]",function(b){var c=a(b.target);c.hasClass("btn")||(c=c.closest(".btn")),c.button("toggle"),b.preventDefault()})}(window.jQuery),+function(a){"use strict";var b=function(b,c){this.$element=a(b),this.$indicators=this.$element.find(".carousel-indicators"),this.options=c,this.paused=this.sliding=this.interval=this.$active=this.$items=null,"hover"==this.options.pause&&this.$element.on("mouseenter",a.proxy(this.pause,this))
 .on("mouseleave",a.proxy(this.cycle,this))};b.DEFAULTS={interval:5e3,pause:"hover",wrap:!0},b.prototype.cycle=function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},b.prototype.getActiveIndex=function(){return this.$active=this.$element.find(".item.active"),this.$items=this.$active.parent().children(),this.$items.index(this.$active)},b.prototype.to=function(b){var c=this,d=this.getActiveIndex();return b>this.$items.length-1||0>b?void 0:this.sliding?this.$element.one("slid",function(){c.to(b)}):d==b?this.pause().cycle():this.slide(b>d?"next":"prev",a(this.$items[b]))},b.prototype.pause=function(b){return b||(this.paused=!0),this.$element.find(".next, .prev").length&&a.support.transition.end&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),this.interval=clearInterval(this.interval),this},b.prototype.next=function(){return th
 is.sliding?void 0:this.slide("next")},b.prototype.prev=function(){return this.sliding?void 0:this.slide("prev")},b.prototype.slide=function(b,c){var d=this.$element.find(".item.active"),e=c||d[b](),f=this.interval,g="next"==b?"left":"right",h="next"==b?"first":"last",i=this;if(!e.length){if(!this.options.wrap)return;e=this.$element.find(".item")[h]()}this.sliding=!0,f&&this.pause();var j=a.Event("slide.bs.carousel",{relatedTarget:e[0],direction:g});if(!e.hasClass("active")){if(this.$indicators.length&&(this.$indicators.find(".active").removeClass("active"),this.$element.one("slid",function(){var b=a(i.$indicators.children()[i.getActiveIndex()]);b&&b.addClass("active")})),a.support.transition&&this.$element.hasClass("slide")){if(this.$element.trigger(j),j.isDefaultPrevented())return;e.addClass(b),e[0].offsetWidth,d.addClass(g),e.addClass(g),d.one(a.support.transition.end,function(){e.removeClass([b,g].join(" ")).addClass("active"),d.removeClass(["active",g].join(" ")),i.sliding=!1,se
 tTimeout(function(){i.$element.trigger("slid")},0)}).emulateTransitionEnd(600)}else{if(this.$element.trigger(j),j.isDefaultPrevented())return;d.removeClass("active"),e.addClass("active"),this.sliding=!1,this.$element.trigger("slid")}return f&&this.cycle(),this}};var c=a.fn.carousel;a.fn.carousel=function(c){return this.each(function(){var d=a(this),e=d.data("bs.carousel"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c),g="string"==typeof c?c:f.slide;e||d.data("bs.carousel",e=new b(this,f)),"number"==typeof c?e.to(c):g?e[g]():f.interval&&e.pause().cycle()})},a.fn.carousel.Constructor=b,a.fn.carousel.noConflict=function(){return a.fn.carousel=c,this},a(document).on("click.bs.carousel.data-api","[data-slide], [data-slide-to]",function(b){var c,d=a(this),e=a(d.attr("data-target")||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,"")),f=a.extend({},e.data(),d.data()),g=d.attr("data-slide-to");g&&(f.interval=!1),e.carousel(f),(g=d.attr("data-slide-to"))&&e.data("bs.carousel").to(g)
 ,b.preventDefault()}),a(window).on("load",function(){a('[data-ride="carousel"]').each(function(){var b=a(this);b.carousel(b.data())})})}(window.jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d),this.transitioning=null,this.options.parent&&(this.$parent=a(this.options.parent)),this.options.toggle&&this.toggle()};b.DEFAULTS={toggle:!0},b.prototype.dimension=function(){var a=this.$element.hasClass("width");return a?"width":"height"},b.prototype.show=function(){if(!this.transitioning&&!this.$element.hasClass("in")){var b=a.Event("show.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.$parent&&this.$parent.find("> .panel > .in");if(c&&c.length){var d=c.data("bs.collapse");if(d&&d.transitioning)return;c.collapse("hide"),d||c.data("bs.collapse",null)}var e=this.dimension();this.$element.removeClass("collapse").addClass("collapsing")[e](0),this.transitioning=1;var f=function(){this.$element.removeClass
 ("collapsing").addClass("in")[e]("auto"),this.transitioning=0,this.$element.trigger("shown.bs.collapse")};if(!a.support.transition)return f.call(this);var g=a.camelCase(["scroll",e].join("-"));this.$element.one(a.support.transition.end,a.proxy(f,this)).emulateTransitionEnd(350)[e](this.$element[0][g])}}},b.prototype.hide=function(){if(!this.transitioning&&this.$element.hasClass("in")){var b=a.Event("hide.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.dimension();this.$element[c](this.$element[c]())[0].offsetHeight,this.$element.addClass("collapsing").removeClass("collapse").removeClass("in"),this.transitioning=1;var d=function(){this.transitioning=0,this.$element.trigger("hidden.bs.collapse").removeClass("collapsing").addClass("collapse")};return a.support.transition?(this.$element[c](0).one(a.support.transition.end,a.proxy(d,this)).emulateTransitionEnd(350),void 0):d.call(this)}}},b.prototype.toggle=function(){this[this.$element.hasClass("in")?"hide":
 "show"]()};var c=a.fn.collapse;a.fn.collapse=function(c){return this.each(function(){var d=a(this),e=d.data("bs.collapse"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c);e||d.data("bs.collapse",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.collapse.Constructor=b,a.fn.collapse.noConflict=function(){return a.fn.collapse=c,this},a(document).on("click.bs.collapse.data-api","[data-toggle=collapse]",function(b){var c,d=a(this),e=d.attr("data-target")||b.preventDefault()||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,""),f=a(e),g=f.data("bs.collapse"),h=g?"toggle":d.data(),i=d.attr("data-parent"),j=i&&a(i);g&&g.transitioning||(j&&j.find('[data-toggle=collapse][data-parent="'+i+'"]').not(d).addClass("collapsed"),d[f.hasClass("in")?"addClass":"removeClass"]("collapsed")),f.collapse(h)})}(window.jQuery),+function(a){"use strict";function b(){a(d).remove(),a(e).each(function(b){var d=c(a(this));d.hasClass("open")&&(d.trigger(b=a.Event("hide.bs.dropdown")),b.isDefaultPrevented(
 )||d.removeClass("open").trigger("hidden.bs.dropdown"))})}function c(b){var c=b.attr("data-target");c||(c=b.attr("href"),c=c&&/#/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,""));var d=c&&a(c);return d&&d.length?d:b.parent()}var d=".dropdown-backdrop",e="[data-toggle=dropdown]",f=function(b){a(b).on("click.bs.dropdown",this.toggle)};f.prototype.toggle=function(d){var e=a(this);if(!e.is(".disabled, :disabled")){var f=c(e),g=f.hasClass("open");if(b(),!g){if("ontouchstart"in document.documentElement&&!f.closest(".navbar-nav").length&&a('<div class="dropdown-backdrop"/>').insertAfter(a(this)).on("click",b),f.trigger(d=a.Event("show.bs.dropdown")),d.isDefaultPrevented())return;f.toggleClass("open").trigger("shown.bs.dropdown"),e.focus()}return!1}},f.prototype.keydown=function(b){if(/(38|40|27)/.test(b.keyCode)){var d=a(this);if(b.preventDefault(),b.stopPropagation(),!d.is(".disabled, :disabled")){var f=c(d),g=f.hasClass("open");if(!g||g&&27==b.keyCode)return 27==b.which&&f.find(e).focus(),d.clic
 k();var h=a("[role=menu] li:not(.divider):visible a",f);if(h.length){var i=h.index(h.filter(":focus"));38==b.keyCode&&i>0&&i--,40==b.keyCode&&i<h.length-1&&i++,~i||(i=0),h.eq(i).focus()}}}};var g=a.fn.dropdown;a.fn.dropdown=function(b){return this.each(function(){var c=a(this),d=c.data("dropdown");d||c.data("dropdown",d=new f(this)),"string"==typeof b&&d[b].call(c)})},a.fn.dropdown.Constructor=f,a.fn.dropdown.noConflict=function(){return a.fn.dropdown=g,this},a(document).on("click.bs.dropdown.data-api",b).on("click.bs.dropdown.data-api",".dropdown form",function(a){a.stopPropagation()}).on("click.bs.dropdown.data-api",e,f.prototype.toggle).on("keydown.bs.dropdown.data-api",e+", [role=menu]",f.prototype.keydown)}(window.jQuery),+function(a){"use strict";var b=function(b,c){this.options=c,this.$element=a(b),this.$backdrop=this.isShown=null,this.options.remote&&this.$element.load(this.options.remote)};b.DEFAULTS={backdrop:!0,keyboard:!0,show:!0},b.prototype.toggle=function(a){return th
 is[this.isShown?"hide":"show"](a)},b.prototype.show=function(b){var c=this,d=a.Event("show.bs.modal",{relatedTarget:b});this.$element.trigger(d),this.isShown||d.isDefaultPrevented()||(this.isShown=!0,this.escape(),this.$element.on("click.dismiss.modal",'[data-dismiss="modal"]',a.proxy(this.hide,this)),this.backdrop(function(){var d=a.support.transition&&c.$element.hasClass("fade");c.$element.parent().length||c.$element.appendTo(document.body),c.$element.show(),d&&c.$element[0].offsetWidth,c.$element.addClass("in").attr("aria-hidden",!1),c.enforceFocus();var e=a.Event("shown.bs.modal",{relatedTarget:b});d?c.$element.find(".modal-dialog").one(a.support.transition.end,function(){c.$element.focus().trigger(e)}).emulateTransitionEnd(300):c.$element.focus().trigger(e)}))},b.prototype.hide=function(b){b&&b.preventDefault(),b=a.Event("hide.bs.modal"),this.$element.trigger(b),this.isShown&&!b.isDefaultPrevented()&&(this.isShown=!1,this.escape(),a(document).off("focusin.bs.modal"),this.$eleme
 nt.removeClass("in").attr("aria-hidden",!0).off("click.dismiss.modal"),a.support.transition&&this.$element.hasClass("fade")?this.$element.one(a.support.transition.end,a.proxy(this.hideModal,this)).emulateTransitionEnd(300):this.hideModal())},b.prototype.enforceFocus=function(){a(document).off("focusin.bs.modal").on("focusin.bs.modal",a.proxy(function(a){this.$element[0]===a.target||this.$element.has(a.target).length||this.$element.focus()},this))},b.prototype.escape=function(){this.isShown&&this.options.keyboard?this.$element.on("keyup.dismiss.bs.modal",a.proxy(function(a){27==a.which&&this.hide()},this)):this.isShown||this.$element.off("keyup.dismiss.bs.modal")},b.prototype.hideModal=function(){var a=this;this.$element.hide(),this.backdrop(function(){a.removeBackdrop(),a.$element.trigger("hidden.bs.modal")})},b.prototype.removeBackdrop=function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},b.prototype.backdrop=function(b){var c=this.$element.hasClass("fade")?"fade"
 :"";if(this.isShown&&this.options.backdrop){var d=a.support.transition&&c;if(this.$backdrop=a('<div class="modal-backdrop '+c+'" />').appendTo(document.body),this.$element.on("click.dismiss.modal",a.proxy(function(a){a.target===a.currentTarget&&("static"==this.options.backdrop?this.$element[0].focus.call(this.$element[0]):this.hide.call(this))},this)),d&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in"),!b)return;d?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()}else!this.isShown&&this.$backdrop?(this.$backdrop.removeClass("in"),a.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()):b&&b()};var c=a.fn.modal;a.fn.modal=function(c,d){return this.each(function(){var e=a(this),f=e.data("bs.modal"),g=a.extend({},b.DEFAULTS,e.data(),"object"==typeof c&&c);f||e.data("bs.modal",f=new b(this,g)),"string"==typeof c?f[c](d):g.show&&f.show(d)})},a.fn.modal.Constructor=b,a.fn.modal.
 noConflict=function(){return a.fn.modal=c,this},a(document).on("click.bs.modal.data-api",'[data-toggle="modal"]',function(b){var c=a(this),d=c.attr("href"),e=a(c.attr("data-target")||d&&d.replace(/.*(?=#[^\s]+$)/,"")),f=e.data("modal")?"toggle":a.extend({remote:!/#/.test(d)&&d},e.data(),c.data());b.preventDefault(),e.modal(f,this).one("hide",function(){c.is(":visible")&&c.focus()})}),a(document).on("show.bs.modal",".modal",function(){a(document.body).addClass("modal-open")}).on("hidden.bs.modal",".modal",function(){a(document.body).removeClass("modal-open")})}(window.jQuery),+function(a){"use strict";var b=function(a,b){this.type=this.options=this.enabled=this.timeout=this.hoverState=this.$element=null,this.init("tooltip",a,b)};b.DEFAULTS={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1},b.prototype.init=function(b,c,d){this.en
 abled=!0,this.type=b,this.$element=a(c),this.options=this.getOptions(d);for(var e=this.options.trigger.split(" "),f=e.length;f--;){var g=e[f];if("click"==g)this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this));else if("manual"!=g){var h="hover"==g?"mouseenter":"focus",i="hover"==g?"mouseleave":"blur";this.$element.on(h+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(i+"."+this.type,this.options.selector,a.proxy(this.leave,this))}}this.options.selector?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.getOptions=function(b){return b=a.extend({},this.getDefaults(),this.$element.data(),b),b.delay&&"number"==typeof b.delay&&(b.delay={show:b.delay,hide:b.delay}),b},b.prototype.getDelegateOptions=function(){var b={},c=this.getDefaults();return this._options&&a.each(this._options,function(a,d){c[a]!=d&&(b[a]=d)}),b},b.prototyp
 e.enter=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="in",c.options.delay&&c.options.delay.show?(c.timeout=setTimeout(function(){"in"==c.hoverState&&c.show()},c.options.delay.show),void 0):c.show()},b.prototype.leave=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="out",c.options.delay&&c.options.delay.hide?(c.timeout=setTimeout(function(){"out"==c.hoverState&&c.hide()},c.options.delay.hide),void 0):c.hide()},b.prototype.show=function(){var b=a.Event("show.bs."+this.type);if(this.hasContent()&&this.enabled){if(this.$element.trigger(b),b.isDefaultPrevented())return;var c=this.tip();this.setContent(),this.options.animation&&c.addClass("fade");var d="function"==typeof this.options.placement?this.options.placement.call(this,c[0],this.$ele
 ment[0]):this.options.placement,e=/\s?auto?\s?/i,f=e.test(d);f&&(d=d.replace(e,"")||"top"),c.detach().css({top:0,left:0,display:"block"}).addClass(d),this.options.container?c.appendTo(this.options.container):c.insertAfter(this.$element);var g=this.getPosition(),h=c[0].offsetWidth,i=c[0].offsetHeight;if(f){var j=this.$element.parent(),k=d,l=document.documentElement.scrollTop||document.body.scrollTop,m="body"==this.options.container?window.innerWidth:j.outerWidth(),n="body"==this.options.container?window.innerHeight:j.outerHeight(),o="body"==this.options.container?0:j.offset().left;d="bottom"==d&&g.top+g.height+i-l>n?"top":"top"==d&&g.top-l-i<0?"bottom":"right"==d&&g.right+h>m?"left":"left"==d&&g.left-h<o?"right":d,c.removeClass(k).addClass(d)}var p=this.getCalculatedOffset(d,g,h,i);this.applyPlacement(p,d),this.$element.trigger("shown.bs."+this.type)}},b.prototype.applyPlacement=function(a,b){var c,d=this.tip(),e=d[0].offsetWidth,f=d[0].offsetHeight,g=parseInt(d.css("margin-top"),10)
 ,h=parseInt(d.css("margin-left"),10);isNaN(g)&&(g=0),isNaN(h)&&(h=0),a.top=a.top+g,a.left=a.left+h,d.offset(a).addClass("in");var i=d[0].offsetWidth,j=d[0].offsetHeight;if("top"==b&&j!=f&&(c=!0,a.top=a.top+f-j),/bottom|top/.test(b)){var k=0;a.left<0&&(k=-2*a.left,a.left=0,d.offset(a),i=d[0].offsetWidth,j=d[0].offsetHeight),this.replaceArrow(k-e+i,i,"left")}else this.replaceArrow(j-f,j,"top");c&&d.offset(a)},b.prototype.replaceArrow=function(a,b,c){this.arrow().css(c,a?50*(1-a/b)+"%":"")},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle();a.find(".tooltip-inner")[this.options.html?"html":"text"](b),a.removeClass("fade in top bottom left right")},b.prototype.hide=function(){function b(){"in"!=c.hoverState&&d.detach()}var c=this,d=this.tip(),e=a.Event("hide.bs."+this.type);return this.$element.trigger(e),e.isDefaultPrevented()?void 0:(d.removeClass("in"),a.support.transition&&this.$tip.hasClass("fade")?d.one(a.support.transition.end,b).emulateTransitionEnd(150):b(),th
 is.$element.trigger("hidden.bs."+this.type),this)},b.prototype.fixTitle=function(){var a=this.$element;(a.attr("title")||"string"!=typeof a.attr("data-original-title"))&&a.attr("data-original-title",a.attr("title")||"").attr("title","")},b.prototype.hasContent=function(){return this.getTitle()},b.prototype.getPosition=function(){var b=this.$element[0];return a.extend({},"function"==typeof b.getBoundingClientRect?b.getBoundingClientRect():{width:b.offsetWidth,height:b.offsetHeight},this.$element.offset())},b.prototype.getCalculatedOffset=function(a,b,c,d){return"bottom"==a?{top:b.top+b.height,left:b.left+b.width/2-c/2}:"top"==a?{top:b.top-d,left:b.left+b.width/2-c/2}:"left"==a?{top:b.top+b.height/2-d/2,left:b.left-c}:{top:b.top+b.height/2-d/2,left:b.left+b.width}},b.prototype.getTitle=function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||("function"==typeof c.title?c.title.call(b[0]):c.title)},b.prototype.tip=function(){return this.$tip=this.$tip||a(
 this.options.template)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},b.prototype.validate=function(){this.$element[0].parentNode||(this.hide(),this.$element=null,this.options=null)},b.prototype.enable=function(){this.enabled=!0},b.prototype.disable=function(){this.enabled=!1},b.prototype.toggleEnabled=function(){this.enabled=!this.enabled},b.prototype.toggle=function(b){var c=b?a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type):this;c.tip().hasClass("in")?c.leave(c):c.enter(c)},b.prototype.destroy=function(){this.hide().$element.off("."+this.type).removeData("bs."+this.type)};var c=a.fn.tooltip;a.fn.tooltip=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tooltip"),f="object"==typeof c&&c;e||d.data("bs.tooltip",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.tooltip.Constructor=b,a.fn.tooltip.noConflict=function(){return a.fn.tooltip=c,this}}(window.jQuery),+function(a){"use strict";
 var b=function(a,b){this.init("popover",a,b)};if(!a.fn.tooltip)throw new Error("Popover requires tooltip.js");b.DEFAULTS=a.extend({},a.fn.tooltip.Constructor.DEFAULTS,{placement:"right",trigger:"click",content:"",template:'<div class="popover"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),b.prototype=a.extend({},a.fn.tooltip.Constructor.prototype),b.prototype.constructor=b,b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content")[this.options.html?"html":"text"](c),a.removeClass("fade top bottom left right in"),a.find(".popover-title").html()||a.find(".popover-title").hide()},b.prototype.hasContent=function(){return this.getTitle()||this.getContent()},b.prototype.getContent=function(){var a=this.$element,b=this.options;return a.attr("data-content")||("functio
 n"==typeof b.content?b.content.call(a[0]):b.content)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".arrow")},b.prototype.tip=function(){return this.$tip||(this.$tip=a(this.options.template)),this.$tip};var c=a.fn.popover;a.fn.popover=function(c){return this.each(function(){var d=a(this),e=d.data("bs.popover"),f="object"==typeof c&&c;e||d.data("bs.popover",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.popover.Constructor=b,a.fn.popover.noConflict=function(){return a.fn.popover=c,this}}(window.jQuery),+function(a){"use strict";function b(c,d){var e,f=a.proxy(this.process,this);this.$element=a(c).is("body")?a(window):a(c),this.$body=a("body"),this.$scrollElement=this.$element.on("scroll.bs.scroll-spy.data-api",f),this.options=a.extend({},b.DEFAULTS,d),this.selector=(this.options.target||(e=a(c).attr("href"))&&e.replace(/.*(?=#[^\s]+$)/,"")||"")+" .nav li > a",this.offsets=a([]),this.targets=a([]),this.activeTarget=null,this.refresh(),this.process(
 )}b.DEFAULTS={offset:10},b.prototype.refresh=function(){var b=this.$element[0]==window?"offset":"position";this.offsets=a([]),this.targets=a([]);var c=this;this.$body.find(this.selector).map(function(){var d=a(this),e=d.data("target")||d.attr("href"),f=/^#\w/.test(e)&&a(e);return f&&f.length&&[[f[b]().top+(!a.isWindow(c.$scrollElement.get(0))&&c.$scrollElement.scrollTop()),e]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){c.offsets.push(this[0]),c.targets.push(this[1])})},b.prototype.process=function(){var a,b=this.$scrollElement.scrollTop()+this.options.offset,c=this.$scrollElement[0].scrollHeight||this.$body[0].scrollHeight,d=c-this.$scrollElement.height(),e=this.offsets,f=this.targets,g=this.activeTarget;if(b>=d)return g!=(a=f.last()[0])&&this.activate(a);for(a=e.length;a--;)g!=f[a]&&b>=e[a]&&(!e[a+1]||b<=e[a+1])&&this.activate(f[a])},b.prototype.activate=function(b){this.activeTarget=b,a(this.selector).parents(".active").removeClass("active");var c=this.selector+
 '[data-target="'+b+'"],'+this.selector+'[href="'+b+'"]',d=a(c).parents("li").addClass("active");d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate")};var c=a.fn.scrollspy;a.fn.scrollspy=function(c){return this.each(function(){var d=a(this),e=d.data("bs.scrollspy"),f="object"==typeof c&&c;e||d.data("bs.scrollspy",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=c,this},a(window).on("load",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);b.scrollspy(b.data())})})}(window.jQuery),+function(a){"use strict";var b=function(b){this.element=a(b)};b.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.attr("data-target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a")[0],f=a.Event("show.bs.tab",{relatedTarget:e});if(b.trigger(f
 ),!f.isDefaultPrevented()){var g=a(d);this.activate(b.parent("li"),c),this.activate(g,g.parent(),function(){b.trigger({type:"shown.bs.tab",relatedTarget:e})})}}},b.prototype.activate=function(b,c,d){function e(){f.removeClass("active").find("> .dropdown-menu > .active").removeClass("active"),b.addClass("active"),g?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu")&&b.closest("li.dropdown").addClass("active"),d&&d()}var f=c.find("> .active"),g=d&&a.support.transition&&f.hasClass("fade");g?f.one(a.support.transition.end,e).emulateTransitionEnd(150):e(),f.removeClass("in")};var c=a.fn.tab;a.fn.tab=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new b(this)),"string"==typeof c&&e[c]()})},a.fn.tab.Constructor=b,a.fn.tab.noConflict=function(){return a.fn.tab=c,this},a(document).on("click.bs.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(b){b.preventDefault(),a(this).tab("show")})}(window.jQ
 uery),+function(a){"use strict";var b=function(c,d){this.options=a.extend({},b.DEFAULTS,d),this.$window=a(window).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(c),this.affixed=this.unpin=null,this.checkPosition()};b.RESET="affix affix-top affix-bottom",b.DEFAULTS={offset:0},b.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},b.prototype.checkPosition=function(){if(this.$element.is(":visible")){var c=a(document).height(),d=this.$window.scrollTop(),e=this.$element.offset(),f=this.options.offset,g=f.top,h=f.bottom;"object"!=typeof f&&(h=g=f),"function"==typeof g&&(g=f.top()),"function"==typeof h&&(h=f.bottom());var i=null!=this.unpin&&d+this.unpin<=e.top?!1:null!=h&&e.top+this.$element.height()>=c-h?"bottom":null!=g&&g>=d?"top":!1;this.affixed!==i&&(this.unpin&&this.$element.css("top",""),this.affixed=i,this.unpin="bottom"==i?e.top-
 d:null,this.$element.removeClass(b.RESET).addClass("affix"+(i?"-"+i:"")),"bottom"==i&&this.$element.offset({top:document.body.offsetHeight-h-this.$element.height()}))}};var c=a.fn.affix;a.fn.affix=function(c){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof c&&c;e||d.data("bs.affix",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.affix.Constructor=b,a.fn.affix.noConflict=function(){return a.fn.affix=c,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var b=a(this),c=b.data();c.offset=c.offset||{},c.offsetBottom&&(c.offset.bottom=c.offsetBottom),c.offsetTop&&(c.offset.top=c.offsetTop),b.affix(c)})})}(window.jQuery);
\ No newline at end of file
+if(!jQuery)throw new Error("Bootstrap requires jQuery");+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]}}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one(a.support.transition.end,function(){c=!0});var e=function(){c||a(d).trigger(a.support.transition.end)};return setTimeout(e,b),this},a(function(){a.support.transition=b()})}(window.jQuery),+function(a){"use strict";var b='[data-dismiss="alert"]',c=function(c){a(c).on("click",b,this.close)};c.prototype.close=function(b){function c(){f.trigger("closed.bs.alert").remove()}var d=a(this),e=d.attr("data-target");e||(e=d.attr("href"),e=e&&e.replace(/.*(?=#[^\s]*$)/,""));var f=a(e);b&&b.preventDefault(),f.length||(f=d.hasClass("alert")?d:d.parent()),f.trigger(b=a.Event("close.bs.alert")),b.isDefau
 ltPrevented()||(f.removeClass("in"),a.support.transition&&f.hasClass("fade")?f.one(a.support.transition.end,c).emulateTransitionEnd(150):c())};var d=a.fn.alert;a.fn.alert=function(b){return this.each(function(){var d=a(this),e=d.data("bs.alert");e||d.data("bs.alert",e=new c(this)),"string"==typeof b&&e[b].call(d)})},a.fn.alert.Constructor=c,a.fn.alert.noConflict=function(){return a.fn.alert=d,this},a(document).on("click.bs.alert.data-api",b,c.prototype.close)}(window.jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d)};b.DEFAULTS={loadingText:"loading..."},b.prototype.setState=function(a){var b="disabled",c=this.$element,d=c.is("input")?"val":"html",e=c.data();a+="Text",e.resetText||c.data("resetText",c[d]()),c[d](e[a]||this.options[a]),setTimeout(function(){"loadingText"==a?c.addClass(b).attr(b,b):c.removeClass(b).removeAttr(b)},0)},b.prototype.toggle=function(){var a=this.$element.closest('[data-toggle="buttons"]');if(a.l
 ength){var b=this.$element.find("input").prop("checked",!this.$element.hasClass("active")).trigger("change");"radio"===b.prop("type")&&a.find(".active").removeClass("active")}this.$element.toggleClass("active")};var c=a.fn.button;a.fn.button=function(c){return this.each(function(){var d=a(this),e=d.data("bs.button"),f="object"==typeof c&&c;e||d.data("bs.button",e=new b(this,f)),"toggle"==c?e.toggle():c&&e.setState(c)})},a.fn.button.Constructor=b,a.fn.button.noConflict=function(){return a.fn.button=c,this},a(document).on("click.bs.button.data-api","[data-toggle^=button]",function(b){var c=a(b.target);c.hasClass("btn")||(c=c.closest(".btn")),c.button("toggle"),b.preventDefault()})}(window.jQuery),+function(a){"use strict";var b=function(b,c){this.$element=a(b),this.$indicators=this.$element.find(".carousel-indicators"),this.options=c,this.paused=this.sliding=this.interval=this.$active=this.$items=null,"hover"==this.options.pause&&this.$element.on("mouseenter",a.proxy(this.pause,this))
 .on("mouseleave",a.proxy(this.cycle,this))};b.DEFAULTS={interval:5e3,pause:"hover",wrap:!0},b.prototype.cycle=function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},b.prototype.getActiveIndex=function(){return this.$active=this.$element.find(".item.active"),this.$items=this.$active.parent().children(),this.$items.index(this.$active)},b.prototype.to=function(b){var c=this,d=this.getActiveIndex();return b>this.$items.length-1||0>b?void 0:this.sliding?this.$element.one("slid",function(){c.to(b)}):d==b?this.pause().cycle():this.slide(b>d?"next":"prev",a(this.$items[b]))},b.prototype.pause=function(b){return b||(this.paused=!0),this.$element.find(".next, .prev").length&&a.support.transition.end&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),this.interval=clearInterval(this.interval),this},b.prototype.next=function(){return th
 is.sliding?void 0:this.slide("next")},b.prototype.prev=function(){return this.sliding?void 0:this.slide("prev")},b.prototype.slide=function(b,c){var d=this.$element.find(".item.active"),e=c||d[b](),f=this.interval,g="next"==b?"left":"right",h="next"==b?"first":"last",i=this;if(!e.length){if(!this.options.wrap)return;e=this.$element.find(".item")[h]()}this.sliding=!0,f&&this.pause();var j=a.Event("slide.bs.carousel",{relatedTarget:e[0],direction:g});if(!e.hasClass("active")){if(this.$indicators.length&&(this.$indicators.find(".active").removeClass("active"),this.$element.one("slid",function(){var b=a(i.$indicators.children()[i.getActiveIndex()]);b&&b.addClass("active")})),a.support.transition&&this.$element.hasClass("slide")){if(this.$element.trigger(j),j.isDefaultPrevented())return;e.addClass(b),e[0].offsetWidth,d.addClass(g),e.addClass(g),d.one(a.support.transition.end,function(){e.removeClass([b,g].join(" ")).addClass("active"),d.removeClass(["active",g].join(" ")),i.sliding=!1,se
 tTimeout(function(){i.$element.trigger("slid")},0)}).emulateTransitionEnd(600)}else{if(this.$element.trigger(j),j.isDefaultPrevented())return;d.removeClass("active"),e.addClass("active"),this.sliding=!1,this.$element.trigger("slid")}return f&&this.cycle(),this}};var c=a.fn.carousel;a.fn.carousel=function(c){return this.each(function(){var d=a(this),e=d.data("bs.carousel"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c),g="string"==typeof c?c:f.slide;e||d.data("bs.carousel",e=new b(this,f)),"number"==typeof c?e.to(c):g?e[g]():f.interval&&e.pause().cycle()})},a.fn.carousel.Constructor=b,a.fn.carousel.noConflict=function(){return a.fn.carousel=c,this},a(document).on("click.bs.carousel.data-api","[data-slide], [data-slide-to]",function(b){var c,d=a(this),e=a(d.attr("data-target")||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,"")),f=a.extend({},e.data(),d.data()),g=d.attr("data-slide-to");g&&(f.interval=!1),e.carousel(f),(g=d.attr("data-slide-to"))&&e.data("bs.carousel").to(g)
 ,b.preventDefault()}),a(window).on("load",function(){a('[data-ride="carousel"]').each(function(){var b=a(this);b.carousel(b.data())})})}(window.jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d),this.transitioning=null,this.options.parent&&(this.$parent=a(this.options.parent)),this.options.toggle&&this.toggle()};b.DEFAULTS={toggle:!0},b.prototype.dimension=function(){var a=this.$element.hasClass("width");return a?"width":"height"},b.prototype.show=function(){if(!this.transitioning&&!this.$element.hasClass("in")){var b=a.Event("show.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.$parent&&this.$parent.find("> .panel > .in");if(c&&c.length){var d=c.data("bs.collapse");if(d&&d.transitioning)return;c.collapse("hide"),d||c.data("bs.collapse",null)}var e=this.dimension();this.$element.removeClass("collapse").addClass("collapsing")[e](0),this.transitioning=1;var f=function(){this.$element.removeClass
 ("collapsing").addClass("in")[e]("auto"),this.transitioning=0,this.$element.trigger("shown.bs.collapse")};if(!a.support.transition)return f.call(this);var g=a.camelCase(["scroll",e].join("-"));this.$element.one(a.support.transition.end,a.proxy(f,this)).emulateTransitionEnd(350)[e](this.$element[0][g])}}},b.prototype.hide=function(){if(!this.transitioning&&this.$element.hasClass("in")){var b=a.Event("hide.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.dimension();this.$element[c](this.$element[c]())[0].offsetHeight,this.$element.addClass("collapsing").removeClass("collapse").removeClass("in"),this.transitioning=1;var d=function(){this.transitioning=0,this.$element.trigger("hidden.bs.collapse").removeClass("collapsing").addClass("collapse")};return a.support.transition?(this.$element[c](0).one(a.support.transition.end,a.proxy(d,this)).emulateTransitionEnd(350),void 0):d.call(this)}}},b.prototype.toggle=function(){this[this.$element.hasClass("in")?"hide":
 "show"]()};var c=a.fn.collapse;a.fn.collapse=function(c){return this.each(function(){var d=a(this),e=d.data("bs.collapse"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c);e||d.data("bs.collapse",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.collapse.Constructor=b,a.fn.collapse.noConflict=function(){return a.fn.collapse=c,this},a(document).on("click.bs.collapse.data-api","[data-toggle=collapse]",function(b){var c,d=a(this),e=d.attr("data-target")||b.preventDefault()||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,""),f=a(e),g=f.data("bs.collapse"),h=g?"toggle":d.data(),i=d.attr("data-parent"),j=i&&a(i);g&&g.transitioning||(j&&j.find('[data-toggle=collapse][data-parent="'+i+'"]').not(d).addClass("collapsed"),d[f.hasClass("in")?"addClass":"removeClass"]("collapsed")),f.collapse(h)})}(window.jQuery),+function(a){"use strict";function b(){a(d).remove(),a(e).each(function(b){var d=c(a(this));d.hasClass("open")&&(d.trigger(b=a.Event("hide.bs.dropdown")),b.isDefaultPrevented(
 )||d.removeClass("open").trigger("hidden.bs.dropdown"))})}function c(b){var c=b.attr("data-target");c||(c=b.attr("href"),c=c&&/#/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,""));var d=c&&a(c);return d&&d.length?d:b.parent()}var d=".dropdown-backdrop",e="[data-toggle=dropdown]",f=function(b){a(b).on("click.bs.dropdown",this.toggle)};f.prototype.toggle=function(d){var e=a(this);if(!e.is(".disabled, :disabled")){var f=c(e),g=f.hasClass("open");if(b(),!g){if("ontouchstart"in document.documentElement&&!f.closest(".navbar-nav").length&&a('<div class="dropdown-backdrop"/>').insertAfter(a(this)).on("click",b),f.trigger(d=a.Event("show.bs.dropdown")),d.isDefaultPrevented())return;f.toggleClass("open").trigger("shown.bs.dropdown"),e.focus()}return!1}},f.prototype.keydown=function(b){if(/(38|40|27)/.test(b.keyCode)){var d=a(this);if(b.preventDefault(),b.stopPropagation(),!d.is(".disabled, :disabled")){var f=c(d),g=f.hasClass("open");if(!g||g&&27==b.keyCode)return 27==b.which&&f.find(e).focus(),d.clic
 k();var h=a("[role=menu] li:not(.divider):visible a",f);if(h.length){var i=h.index(h.filter(":focus"));38==b.keyCode&&i>0&&i--,40==b.keyCode&&i<h.length-1&&i++,~i||(i=0),h.eq(i).focus()}}}};var g=a.fn.dropdown;a.fn.dropdown=function(b){return this.each(function(){var c=a(this),d=c.data("dropdown");d||c.data("dropdown",d=new f(this)),"string"==typeof b&&d[b].call(c)})},a.fn.dropdown.Constructor=f,a.fn.dropdown.noConflict=function(){return a.fn.dropdown=g,this},a(document).on("click.bs.dropdown.data-api",b).on("click.bs.dropdown.data-api",".dropdown form",function(a){a.stopPropagation()}).on("click.bs.dropdown.data-api",e,f.prototype.toggle).on("keydown.bs.dropdown.data-api",e+", [role=menu]",f.prototype.keydown)}(window.jQuery),+function(a){"use strict";var b=function(b,c){this.options=c,this.$element=a(b),this.$backdrop=this.isShown=null,this.options.remote&&this.$element.load(this.options.remote)};b.DEFAULTS={backdrop:!0,keyboard:!0,show:!0},b.prototype.toggle=function(a){return th
 is[this.isShown?"hide":"show"](a)},b.prototype.show=function(b){var c=this,d=a.Event("show.bs.modal",{relatedTarget:b});this.$element.trigger(d),this.isShown||d.isDefaultPrevented()||(this.isShown=!0,this.escape(),this.$element.on("click.dismiss.modal",'[data-dismiss="modal"]',a.proxy(this.hide,this)),this.backdrop(function(){var d=a.support.transition&&c.$element.hasClass("fade");c.$element.parent().length||c.$element.appendTo(document.body),c.$element.show(),d&&c.$element[0].offsetWidth,c.$element.addClass("in").attr("aria-hidden",!1),c.enforceFocus();var e=a.Event("shown.bs.modal",{relatedTarget:b});d?c.$element.find(".modal-dialog").one(a.support.transition.end,function(){c.$element.focus().trigger(e)}).emulateTransitionEnd(300):c.$element.focus().trigger(e)}))},b.prototype.hide=function(b){b&&b.preventDefault(),b=a.Event("hide.bs.modal"),this.$element.trigger(b),this.isShown&&!b.isDefaultPrevented()&&(this.isShown=!1,this.escape(),a(document).off("focusin.bs.modal"),this.$eleme
 nt.removeClass("in").attr("aria-hidden",!0).off("click.dismiss.modal"),a.support.transition&&this.$element.hasClass("fade")?this.$element.one(a.support.transition.end,a.proxy(this.hideModal,this)).emulateTransitionEnd(300):this.hideModal())},b.prototype.enforceFocus=function(){a(document).off("focusin.bs.modal").on("focusin.bs.modal",a.proxy(function(a){this.$element[0]===a.target||this.$element.has(a.target).length||this.$element.focus()},this))},b.prototype.escape=function(){this.isShown&&this.options.keyboard?this.$element.on("keyup.dismiss.bs.modal",a.proxy(function(a){27==a.which&&this.hide()},this)):this.isShown||this.$element.off("keyup.dismiss.bs.modal")},b.prototype.hideModal=function(){var a=this;this.$element.hide(),this.backdrop(function(){a.removeBackdrop(),a.$element.trigger("hidden.bs.modal")})},b.prototype.removeBackdrop=function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},b.prototype.backdrop=function(b){var c=this.$element.hasClass("fade")?"fade"
 :"";if(this.isShown&&this.options.backdrop){var d=a.support.transition&&c;if(this.$backdrop=a('<div class="modal-backdrop '+c+'" />').appendTo(document.body),this.$element.on("click.dismiss.modal",a.proxy(function(a){a.target===a.currentTarget&&("static"==this.options.backdrop?this.$element[0].focus.call(this.$element[0]):this.hide.call(this))},this)),d&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in"),!b)return;d?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()}else!this.isShown&&this.$backdrop?(this.$backdrop.removeClass("in"),a.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()):b&&b()};var c=a.fn.modal;a.fn.modal=function(c,d){return this.each(function(){var e=a(this),f=e.data("bs.modal"),g=a.extend({},b.DEFAULTS,e.data(),"object"==typeof c&&c);f||e.data("bs.modal",f=new b(this,g)),"string"==typeof c?f[c](d):g.show&&f.show(d)})},a.fn.modal.Constructor=b,a.fn.modal.
 noConflict=function(){return a.fn.modal=c,this},a(document).on("click.bs.modal.data-api",'[data-toggle="modal"]',function(b){var c=a(this),d=c.attr("href"),e=a(c.attr("data-target")||d&&d.replace(/.*(?=#[^\s]+$)/,"")),f=e.data("modal")?"toggle":a.extend({remote:!/#/.test(d)&&d},e.data(),c.data());b.preventDefault(),e.modal(f,this).one("hide",function(){c.is(":visible")&&c.focus()})}),a(document).on("show.bs.modal",".modal",function(){a(document.body).addClass("modal-open")}).on("hidden.bs.modal",".modal",function(){a(document.body).removeClass("modal-open")})}(window.jQuery),+function(a){"use strict";var b=function(a,b){this.type=this.options=this.enabled=this.timeout=this.hoverState=this.$element=null,this.init("tooltip",a,b)};b.DEFAULTS={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1},b.prototype.init=function(b,c,d){this.en
 abled=!0,this.type=b,this.$element=a(c),this.options=this.getOptions(d);for(var e=this.options.trigger.split(" "),f=e.length;f--;){var g=e[f];if("click"==g)this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this));else if("manual"!=g){var h="hover"==g?"mouseenter":"focus",i="hover"==g?"mouseleave":"blur";this.$element.on(h+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(i+"."+this.type,this.options.selector,a.proxy(this.leave,this))}}this.options.selector?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.getOptions=function(b){return b=a.extend({},this.getDefaults(),this.$element.data(),b),b.delay&&"number"==typeof b.delay&&(b.delay={show:b.delay,hide:b.delay}),b},b.prototype.getDelegateOptions=function(){var b={},c=this.getDefaults();return this._options&&a.each(this._options,function(a,d){c[a]!=d&&(b[a]=d)}),b},b.prototyp
 e.enter=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="in",c.options.delay&&c.options.delay.show?(c.timeout=setTimeout(function(){"in"==c.hoverState&&c.show()},c.options.delay.show),void 0):c.show()},b.prototype.leave=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="out",c.options.delay&&c.options.delay.hide?(c.timeout=setTimeout(function(){"out"==c.hoverState&&c.hide()},c.options.delay.hide),void 0):c.hide()},b.prototype.show=function(){var b=a.Event("show.bs."+this.type);if(this.hasContent()&&this.enabled){if(this.$element.trigger(b),b.isDefaultPrevented())return;var c=this.tip();this.setContent(),this.options.animation&&c.addClass("fade");var d="function"==typeof this.options.placement?this.options.placement.call(this,c[0],this.$ele
 ment[0]):this.options.placement,e=/\s?auto?\s?/i,f=e.test(d);f&&(d=d.replace(e,"")||"top"),c.detach().css({top:0,left:0,display:"block"}).addClass(d),this.options.container?c.appendTo(this.options.container):c.insertAfter(this.$element);var g=this.getPosition(),h=c[0].offsetWidth,i=c[0].offsetHeight;if(f){var j=this.$element.parent(),k=d,l=document.documentElement.scrollTop||document.body.scrollTop,m="body"==this.options.container?window.innerWidth:j.outerWidth(),n="body"==this.options.container?window.innerHeight:j.outerHeight(),o="body"==this.options.container?0:j.offset().left;d="bottom"==d&&g.top+g.height+i-l>n?"top":"top"==d&&g.top-l-i<0?"bottom":"right"==d&&g.right+h>m?"left":"left"==d&&g.left-h<o?"right":d,c.removeClass(k).addClass(d)}var p=this.getCalculatedOffset(d,g,h,i);this.applyPlacement(p,d),this.$element.trigger("shown.bs."+this.type)}},b.prototype.applyPlacement=function(a,b){var c,d=this.tip(),e=d[0].offsetWidth,f=d[0].offsetHeight,g=parseInt(d.css("margin-top"),10)
 ,h=parseInt(d.css("margin-left"),10);isNaN(g)&&(g=0),isNaN(h)&&(h=0),a.top=a.top+g,a.left=a.left+h,d.offset(a).addClass("in");var i=d[0].offsetWidth,j=d[0].offsetHeight;if("top"==b&&j!=f&&(c=!0,a.top=a.top+f-j),/bottom|top/.test(b)){var k=0;a.left<0&&(k=-2*a.left,a.left=0,d.offset(a),i=d[0].offsetWidth,j=d[0].offsetHeight),this.replaceArrow(k-e+i,i,"left")}else this.replaceArrow(j-f,j,"top");c&&d.offset(a)},b.prototype.replaceArrow=function(a,b,c){this.arrow().css(c,a?50*(1-a/b)+"%":"")},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle();a.find(".tooltip-inner")[this.options.html?"html":"text"](b),a.removeClass("fade in top bottom left right")},b.prototype.hide=function(){function b(){"in"!=c.hoverState&&d.detach()}var c=this,d=this.tip(),e=a.Event("hide.bs."+this.type);return this.$element.trigger(e),e.isDefaultPrevented()?void 0:(d.removeClass("in"),a.support.transition&&this.$tip.hasClass("fade")?d.one(a.support.transition.end,b).emulateTransitionEnd(150):b(),th
 is.$element.trigger("hidden.bs."+this.type),this)},b.prototype.fixTitle=function(){var a=this.$element;(a.attr("title")||"string"!=typeof a.attr("data-original-title"))&&a.attr("data-original-title",a.attr("title")||"").attr("title","")},b.prototype.hasContent=function(){return this.getTitle()},b.prototype.getPosition=function(){var b=this.$element[0];return a.extend({},"function"==typeof b.getBoundingClientRect?b.getBoundingClientRect():{width:b.offsetWidth,height:b.offsetHeight},this.$element.offset())},b.prototype.getCalculatedOffset=function(a,b,c,d){return"bottom"==a?{top:b.top+b.height,left:b.left+b.width/2-c/2}:"top"==a?{top:b.top-d,left:b.left+b.width/2-c/2}:"left"==a?{top:b.top+b.height/2-d/2,left:b.left-c}:{top:b.top+b.height/2-d/2,left:b.left+b.width}},b.prototype.getTitle=function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||("function"==typeof c.title?c.title.call(b[0]):c.title)},b.prototype.tip=function(){return this.$tip=this.$tip||a(
 this.options.template)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},b.prototype.validate=function(){this.$element[0].parentNode||(this.hide(),this.$element=null,this.options=null)},b.prototype.enable=function(){this.enabled=!0},b.prototype.disable=function(){this.enabled=!1},b.prototype.toggleEnabled=function(){this.enabled=!this.enabled},b.prototype.toggle=function(b){var c=b?a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type):this;c.tip().hasClass("in")?c.leave(c):c.enter(c)},b.prototype.destroy=function(){this.hide().$element.off("."+this.type).removeData("bs."+this.type)};var c=a.fn.tooltip;a.fn.tooltip=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tooltip"),f="object"==typeof c&&c;e||d.data("bs.tooltip",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.tooltip.Constructor=b,a.fn.tooltip.noConflict=function(){return a.fn.tooltip=c,this}}(window.jQuery),+function(a){"use strict";
 var b=function(a,b){this.init("popover",a,b)};if(!a.fn.tooltip)throw new Error("Popover requires tooltip.js");b.DEFAULTS=a.extend({},a.fn.tooltip.Constructor.DEFAULTS,{placement:"right",trigger:"click",content:"",template:'<div class="popover"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),b.prototype=a.extend({},a.fn.tooltip.Constructor.prototype),b.prototype.constructor=b,b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content")[this.options.html?"html":"text"](c),a.removeClass("fade top bottom left right in"),a.find(".popover-title").html()||a.find(".popover-title").hide()},b.prototype.hasContent=function(){return this.getTitle()||this.getContent()},b.prototype.getContent=function(){var a=this.$element,b=this.options;return a.attr("data-content")||("functio
 n"==typeof b.content?b.content.call(a[0]):b.content)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".arrow")},b.prototype.tip=function(){return this.$tip||(this.$tip=a(this.options.template)),this.$tip};var c=a.fn.popover;a.fn.popover=function(c){return this.each(function(){var d=a(this),e=d.data("bs.popover"),f="object"==typeof c&&c;e||d.data("bs.popover",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.popover.Constructor=b,a.fn.popover.noConflict=function(){return a.fn.popover=c,this}}(window.jQuery),+function(a){"use strict";function b(c,d){var e,f=a.proxy(this.process,this);this.$element=a(c).is("body")?a(window):a(c),this.$body=a("body"),this.$scrollElement=this.$element.on("scroll.bs.scroll-spy.data-api",f),this.options=a.extend({},b.DEFAULTS,d),this.selector=(this.options.target||(e=a(c).attr("href"))&&e.replace(/.*(?=#[^\s]+$)/,"")||"")+" .nav li > a",this.offsets=a([]),this.targets=a([]),this.activeTarget=null,this.refresh(),this.process(
 )}b.DEFAULTS={offset:10},b.prototype.refresh=function(){var b=this.$element[0]==window?"offset":"position";this.offsets=a([]),this.targets=a([]);var c=this;this.$body.find(this.selector).map(function(){var d=a(this),e=d.data("target")||d.attr("href"),f=/^#\w/.test(e)&&a(e);return f&&f.length&&[[f[b]().top+(!a.isWindow(c.$scrollElement.get(0))&&c.$scrollElement.scrollTop()),e]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){c.offsets.push(this[0]),c.targets.push(this[1])})},b.prototype.process=function(){var a,b=this.$scrollElement.scrollTop()+this.options.offset,c=this.$scrollElement[0].scrollHeight||this.$body[0].scrollHeight,d=c-this.$scrollElement.height(),e=this.offsets,f=this.targets,g=this.activeTarget;if(b>=d)return g!=(a=f.last()[0])&&this.activate(a);for(a=e.length;a--;)g!=f[a]&&b>=e[a]&&(!e[a+1]||b<=e[a+1])&&this.activate(f[a])},b.prototype.activate=function(b){this.activeTarget=b,a(this.selector).parents(".active").removeClass("active");var c=this.selector+
 '[data-target="'+b+'"],'+this.selector+'[href="'+b+'"]',d=a(c).parents("li").addClass("active");d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate")};var c=a.fn.scrollspy;a.fn.scrollspy=function(c){return this.each(function(){var d=a(this),e=d.data("bs.scrollspy"),f="object"==typeof c&&c;e||d.data("bs.scrollspy",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=c,this},a(window).on("load",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);b.scrollspy(b.data())})})}(window.jQuery),+function(a){"use strict";var b=function(b){this.element=a(b)};b.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.attr("data-target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a")[0],f=a.Event("show.bs.tab",{relatedTarget:e});if(b.trigger(f
 ),!f.isDefaultPrevented()){var g=a(d);this.activate(b.parent("li"),c),this.activate(g,g.parent(),function(){b.trigger({type:"shown.bs.tab",relatedTarget:e})})}}},b.prototype.activate=function(b,c,d){function e(){f.removeClass("active").find("> .dropdown-menu > .active").removeClass("active"),b.addClass("active"),g?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu")&&b.closest("li.dropdown").addClass("active"),d&&d()}var f=c.find("> .active"),g=d&&a.support.transition&&f.hasClass("fade");g?f.one(a.support.transition.end,e).emulateTransitionEnd(150):e(),f.removeClass("in")};var c=a.fn.tab;a.fn.tab=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new b(this)),"string"==typeof c&&e[c]()})},a.fn.tab.Constructor=b,a.fn.tab.noConflict=function(){return a.fn.tab=c,this},a(document).on("click.bs.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(b){b.preventDefault(),a(this).tab("show")})}(window.jQ
 uery),+function(a){"use strict";var b=function(c,d){this.options=a.extend({},b.DEFAULTS,d),this.$window=a(window).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(c),this.affixed=this.unpin=null,this.checkPosition()};b.RESET="affix affix-top affix-bottom",b.DEFAULTS={offset:0},b.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},b.prototype.checkPosition=function(){if(this.$element.is(":visible")){var c=a(document).height(),d=this.$window.scrollTop(),e=this.$element.offset(),f=this.options.offset,g=f.top,h=f.bottom;"object"!=typeof f&&(h=g=f),"function"==typeof g&&(g=f.top()),"function"==typeof h&&(h=f.bottom());var i=null!=this.unpin&&d+this.unpin<=e.top?!1:null!=h&&e.top+this.$element.height()>=c-h?"bottom":null!=g&&g>=d?"top":!1;this.affixed!==i&&(this.unpin&&this.$element.css("top",""),this.affixed=i,this.unpin="bottom"==i?e.top-
 d:null,this.$element.removeClass(b.RESET).addClass("affix"+(i?"-"+i:"")),"bottom"==i&&this.$element.offset({top:document.body.offsetHeight-h-this.$element.height()}))}};var c=a.fn.affix;a.fn.affix=function(c){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof c&&c;e||d.data("bs.affix",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.affix.Constructor=b,a.fn.affix.noConflict=function(){return a.fn.affix=c,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var b=a(this),c=b.data();c.offset=c.offset||{},c.offsetBottom&&(c.offset.bottom=c.offsetBottom),c.offsetTop&&(c.offset.top=c.offsetTop),b.affix(c)})})}(window.jQuery);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/content/js/usergrid-site.js
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/js/usergrid-site.js b/gemfire-site/website/content/js/usergrid-site.js
index 1a513fa..115768a 100644
--- a/gemfire-site/website/content/js/usergrid-site.js
+++ b/gemfire-site/website/content/js/usergrid-site.js
@@ -1,3 +1,19 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
 var start = 40;
 var end = 210;
 
@@ -45,6 +61,6 @@ $(document).ready(function() {
     //     else
     //         $("#home-logo").css("opacity", 0);
     // });
-  
 
-});
\ No newline at end of file
+
+});

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/content/releases/index.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/content/releases/index.html b/gemfire-site/website/content/releases/index.html
index 15b460f..8ad0b75 100644
--- a/gemfire-site/website/content/releases/index.html
+++ b/gemfire-site/website/content/releases/index.html
@@ -1,3 +1,20 @@
+<!-- Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License. -->
+
 <section class="bf-tagline">
     <div class="container">
     	<div class="row">

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/layouts/header.html
----------------------------------------------------------------------
diff --git a/gemfire-site/website/layouts/header.html b/gemfire-site/website/layouts/header.html
index 8ea572a..b623104 100644
--- a/gemfire-site/website/layouts/header.html
+++ b/gemfire-site/website/layouts/header.html
@@ -1,4 +1,20 @@
 <head>
+  <!-- Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing,
+  software distributed under the License is distributed on an
+  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  KIND, either express or implied.  See the License for the
+  specific language governing permissions and limitations
+  under the License. -->
     <meta charset="utf-8">
     <title>Apache Geode (incubating) — <%= @item[:title] %></title>
     <meta http-equiv="x-ua-compatible" content="ie=edge" />

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/lib/default.rb
----------------------------------------------------------------------
diff --git a/gemfire-site/website/lib/default.rb b/gemfire-site/website/lib/default.rb
index 8adfad9..dababa8 100644
--- a/gemfire-site/website/lib/default.rb
+++ b/gemfire-site/website/lib/default.rb
@@ -1,3 +1,20 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
 # All files in the 'lib' directory will be loaded
 # before nanoc starts compiling.
 include Nanoc::Helpers::Rendering
@@ -16,28 +33,28 @@ end
 
 class FencedCodeBlock < Nanoc3::Filter
   identifier :fenced_code_block
-  
+
   def run(content, params={})
     content.gsub(/(^`{3,}\s*(\S*)\s*$([^`]*)^`{3,}\s*$)+?/m) {|match|
       lang_spec  = $2
       code_block = $3
-      
+
       replacement = ''
-      
+
       replacement << '<pre class="highlight"><code class="language'
-      
+
       if lang_spec && lang_spec.length > 0
         replacement << '-'
         replacement << lang_spec
       end
-      
+
       replacement << '">'
-      
+
       code_block.gsub!("[:backtick:]", "`")
-      
+
       coder = HTMLEntities.new
       replacement << coder.encode(code_block)
       replacement << '</code></pre>'
     }
   end
-end
\ No newline at end of file
+end

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/lib/helpers_.rb
----------------------------------------------------------------------
diff --git a/gemfire-site/website/lib/helpers_.rb b/gemfire-site/website/lib/helpers_.rb
index e69de29..13a8339 100644
--- a/gemfire-site/website/lib/helpers_.rb
+++ b/gemfire-site/website/lib/helpers_.rb
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/lib/pandoc.template
----------------------------------------------------------------------
diff --git a/gemfire-site/website/lib/pandoc.template b/gemfire-site/website/lib/pandoc.template
index 598e2c2..382b091 100644
--- a/gemfire-site/website/lib/pandoc.template
+++ b/gemfire-site/website/lib/pandoc.template
@@ -1,4 +1,4 @@
 <div class="toc">
 	$toc$
 </div>
-$body$
\ No newline at end of file
+$body$

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/nanoc.yaml
----------------------------------------------------------------------
diff --git a/gemfire-site/website/nanoc.yaml b/gemfire-site/website/nanoc.yaml
index ca1b598..764bb74 100644
--- a/gemfire-site/website/nanoc.yaml
+++ b/gemfire-site/website/nanoc.yaml
@@ -1,3 +1,20 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
 # A list of file extensions that nanoc will consider to be textual rather than
 # binary. If an item with an extension not in this list is found,  the file
 # will be considered as binary.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/run.sh
----------------------------------------------------------------------
diff --git a/gemfire-site/website/run.sh b/gemfire-site/website/run.sh
index cf1f043..cecf19e 100755
--- a/gemfire-site/website/run.sh
+++ b/gemfire-site/website/run.sh
@@ -1 +1,18 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
 nanoc autocompile

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/utilities/map-markers.rb
----------------------------------------------------------------------
diff --git a/gemfire-site/website/utilities/map-markers.rb b/gemfire-site/website/utilities/map-markers.rb
index 4adbac0..19e2306 100644
--- a/gemfire-site/website/utilities/map-markers.rb
+++ b/gemfire-site/website/utilities/map-markers.rb
@@ -1,3 +1,20 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
 require 'rest_client'
 require 'csv'
 require 'json/ext'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/utilities/snapshot-apigee.rb
----------------------------------------------------------------------
diff --git a/gemfire-site/website/utilities/snapshot-apigee.rb b/gemfire-site/website/utilities/snapshot-apigee.rb
index 2fc3ebd..4adf0ca 100644
--- a/gemfire-site/website/utilities/snapshot-apigee.rb
+++ b/gemfire-site/website/utilities/snapshot-apigee.rb
@@ -1,3 +1,20 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
 require 'rubygems'
 require 'mechanize'
 require 'anemone'


[18/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/js/jquery-1.10.1.min.js
----------------------------------------------------------------------
diff --git a/gemfire-site/content/js/jquery-1.10.1.min.js b/gemfire-site/content/js/jquery-1.10.1.min.js
deleted file mode 100644
index e407e76..0000000
--- a/gemfire-site/content/js/jquery-1.10.1.min.js
+++ /dev/null
@@ -1,6 +0,0 @@
-/*! jQuery v1.10.1 | (c) 2005, 2013 jQuery Foundation, Inc. | jquery.org/license
-//@ sourceMappingURL=jquery-1.10.1.min.map
-*/
-(function(e,t){var n,r,i=typeof t,o=e.location,a=e.document,s=a.documentElement,l=e.jQuery,u=e.$,c={},p=[],f="1.10.1",d=p.concat,h=p.push,g=p.slice,m=p.indexOf,y=c.toString,v=c.hasOwnProperty,b=f.trim,x=function(e,t){return new x.fn.init(e,t,r)},w=/[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/.source,T=/\S+/g,C=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,N=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,k=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,E=/^[\],:{}\s]*$/,S=/(?:^|:|,)(?:\s*\[)+/g,A=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,j=/"[^"\\\r\n]*"|true|false|null|-?(?:\d+\.|)\d+(?:[eE][+-]?\d+|)/g,D=/^-ms-/,L=/-([\da-z])/gi,H=function(e,t){return t.toUpperCase()},q=function(e){(a.addEventListener||"load"===e.type||"complete"===a.readyState)&&(_(),x.ready())},_=function(){a.addEventListener?(a.removeEventListener("DOMContentLoaded",q,!1),e.removeEventListener("load",q,!1)):(a.detachEvent("onreadystatechange",q),e.detachEvent("onload",q))};x.fn=x.prototype={jquery:f,constructor:x,init:function(e,n,r){var i,o;if(!e)return this;
 if("string"==typeof e){if(i="<"===e.charAt(0)&&">"===e.charAt(e.length-1)&&e.length>=3?[null,e,null]:N.exec(e),!i||!i[1]&&n)return!n||n.jquery?(n||r).find(e):this.constructor(n).find(e);if(i[1]){if(n=n instanceof x?n[0]:n,x.merge(this,x.parseHTML(i[1],n&&n.nodeType?n.ownerDocument||n:a,!0)),k.test(i[1])&&x.isPlainObject(n))for(i in n)x.isFunction(this[i])?this[i](n[i]):this.attr(i,n[i]);return this}if(o=a.getElementById(i[2]),o&&o.parentNode){if(o.id!==i[2])return r.find(e);this.length=1,this[0]=o}return this.context=a,this.selector=e,this}return e.nodeType?(this.context=this[0]=e,this.length=1,this):x.isFunction(e)?r.ready(e):(e.selector!==t&&(this.selector=e.selector,this.context=e.context),x.makeArray(e,this))},selector:"",length:0,toArray:function(){return g.call(this)},get:function(e){return null==e?this.toArray():0>e?this[this.length+e]:this[e]},pushStack:function(e){var t=x.merge(this.constructor(),e);return t.prevObject=this,t.context=this.context,t},each:function(e,t){retur
 n x.each(this,e,t)},ready:function(e){return x.ready.promise().done(e),this},slice:function(){return this.pushStack(g.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(e){var t=this.length,n=+e+(0>e?t:0);return this.pushStack(n>=0&&t>n?[this[n]]:[])},map:function(e){return this.pushStack(x.map(this,function(t,n){return e.call(t,n,t)}))},end:function(){return this.prevObject||this.constructor(null)},push:h,sort:[].sort,splice:[].splice},x.fn.init.prototype=x.fn,x.extend=x.fn.extend=function(){var e,n,r,i,o,a,s=arguments[0]||{},l=1,u=arguments.length,c=!1;for("boolean"==typeof s&&(c=s,s=arguments[1]||{},l=2),"object"==typeof s||x.isFunction(s)||(s={}),u===l&&(s=this,--l);u>l;l++)if(null!=(o=arguments[l]))for(i in o)e=s[i],r=o[i],s!==r&&(c&&r&&(x.isPlainObject(r)||(n=x.isArray(r)))?(n?(n=!1,a=e&&x.isArray(e)?e:[]):a=e&&x.isPlainObject(e)?e:{},s[i]=x.extend(c,a,r)):r!==t&&(s[i]=r));return s},x.extend({expando:"jQuery"+(f+Math.ran
 dom()).replace(/\D/g,""),noConflict:function(t){return e.$===x&&(e.$=u),t&&e.jQuery===x&&(e.jQuery=l),x},isReady:!1,readyWait:1,holdReady:function(e){e?x.readyWait++:x.ready(!0)},ready:function(e){if(e===!0?!--x.readyWait:!x.isReady){if(!a.body)return setTimeout(x.ready);x.isReady=!0,e!==!0&&--x.readyWait>0||(n.resolveWith(a,[x]),x.fn.trigger&&x(a).trigger("ready").off("ready"))}},isFunction:function(e){return"function"===x.type(e)},isArray:Array.isArray||function(e){return"array"===x.type(e)},isWindow:function(e){return null!=e&&e==e.window},isNumeric:function(e){return!isNaN(parseFloat(e))&&isFinite(e)},type:function(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?c[y.call(e)]||"object":typeof e},isPlainObject:function(e){var n;if(!e||"object"!==x.type(e)||e.nodeType||x.isWindow(e))return!1;try{if(e.constructor&&!v.call(e,"constructor")&&!v.call(e.constructor.prototype,"isPrototypeOf"))return!1}catch(r){return!1}if(x.support.ownLast)for(n in e)return v.call(e,n);fo
 r(n in e);return n===t||v.call(e,n)},isEmptyObject:function(e){var t;for(t in e)return!1;return!0},error:function(e){throw Error(e)},parseHTML:function(e,t,n){if(!e||"string"!=typeof e)return null;"boolean"==typeof t&&(n=t,t=!1),t=t||a;var r=k.exec(e),i=!n&&[];return r?[t.createElement(r[1])]:(r=x.buildFragment([e],t,i),i&&x(i).remove(),x.merge([],r.childNodes))},parseJSON:function(n){return e.JSON&&e.JSON.parse?e.JSON.parse(n):null===n?n:"string"==typeof n&&(n=x.trim(n),n&&E.test(n.replace(A,"@").replace(j,"]").replace(S,"")))?Function("return "+n)():(x.error("Invalid JSON: "+n),t)},parseXML:function(n){var r,i;if(!n||"string"!=typeof n)return null;try{e.DOMParser?(i=new DOMParser,r=i.parseFromString(n,"text/xml")):(r=new ActiveXObject("Microsoft.XMLDOM"),r.async="false",r.loadXML(n))}catch(o){r=t}return r&&r.documentElement&&!r.getElementsByTagName("parsererror").length||x.error("Invalid XML: "+n),r},noop:function(){},globalEval:function(t){t&&x.trim(t)&&(e.execScript||function(t)
 {e.eval.call(e,t)})(t)},camelCase:function(e){return e.replace(D,"ms-").replace(L,H)},nodeName:function(e,t){return e.nodeName&&e.nodeName.toLowerCase()===t.toLowerCase()},each:function(e,t,n){var r,i=0,o=e.length,a=M(e);if(n){if(a){for(;o>i;i++)if(r=t.apply(e[i],n),r===!1)break}else for(i in e)if(r=t.apply(e[i],n),r===!1)break}else if(a){for(;o>i;i++)if(r=t.call(e[i],i,e[i]),r===!1)break}else for(i in e)if(r=t.call(e[i],i,e[i]),r===!1)break;return e},trim:b&&!b.call("\ufeff\u00a0")?function(e){return null==e?"":b.call(e)}:function(e){return null==e?"":(e+"").replace(C,"")},makeArray:function(e,t){var n=t||[];return null!=e&&(M(Object(e))?x.merge(n,"string"==typeof e?[e]:e):h.call(n,e)),n},inArray:function(e,t,n){var r;if(t){if(m)return m.call(t,e,n);for(r=t.length,n=n?0>n?Math.max(0,r+n):n:0;r>n;n++)if(n in t&&t[n]===e)return n}return-1},merge:function(e,n){var r=n.length,i=e.length,o=0;if("number"==typeof r)for(;r>o;o++)e[i++]=n[o];else while(n[o]!==t)e[i++]=n[o++];return e.length
 =i,e},grep:function(e,t,n){var r,i=[],o=0,a=e.length;for(n=!!n;a>o;o++)r=!!t(e[o],o),n!==r&&i.push(e[o]);return i},map:function(e,t,n){var r,i=0,o=e.length,a=M(e),s=[];if(a)for(;o>i;i++)r=t(e[i],i,n),null!=r&&(s[s.length]=r);else for(i in e)r=t(e[i],i,n),null!=r&&(s[s.length]=r);return d.apply([],s)},guid:1,proxy:function(e,n){var r,i,o;return"string"==typeof n&&(o=e[n],n=e,e=o),x.isFunction(e)?(r=g.call(arguments,2),i=function(){return e.apply(n||this,r.concat(g.call(arguments)))},i.guid=e.guid=e.guid||x.guid++,i):t},access:function(e,n,r,i,o,a,s){var l=0,u=e.length,c=null==r;if("object"===x.type(r)){o=!0;for(l in r)x.access(e,n,l,r[l],!0,a,s)}else if(i!==t&&(o=!0,x.isFunction(i)||(s=!0),c&&(s?(n.call(e,i),n=null):(c=n,n=function(e,t,n){return c.call(x(e),n)})),n))for(;u>l;l++)n(e[l],r,s?i:i.call(e[l],l,n(e[l],r)));return o?e:c?n.call(e):u?n(e[0],r):a},now:function(){return(new Date).getTime()},swap:function(e,t,n,r){var i,o,a={};for(o in t)a[o]=e.style[o],e.style[o]=t[o];i=n.apply
 (e,r||[]);for(o in t)e.style[o]=a[o];return i}}),x.ready.promise=function(t){if(!n)if(n=x.Deferred(),"complete"===a.readyState)setTimeout(x.ready);else if(a.addEventListener)a.addEventListener("DOMContentLoaded",q,!1),e.addEventListener("load",q,!1);else{a.attachEvent("onreadystatechange",q),e.attachEvent("onload",q);var r=!1;try{r=null==e.frameElement&&a.documentElement}catch(i){}r&&r.doScroll&&function o(){if(!x.isReady){try{r.doScroll("left")}catch(e){return setTimeout(o,50)}_(),x.ready()}}()}return n.promise(t)},x.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(e,t){c["[object "+t+"]"]=t.toLowerCase()});function M(e){var t=e.length,n=x.type(e);return x.isWindow(e)?!1:1===e.nodeType&&t?!0:"array"===n||"function"!==n&&(0===t||"number"==typeof t&&t>0&&t-1 in e)}r=x(a),function(e,t){var n,r,i,o,a,s,l,u,c,p,f,d,h,g,m,y,v,b="sizzle"+-new Date,w=e.document,T=0,C=0,N=lt(),k=lt(),E=lt(),S=!1,A=function(){return 0},j=typeof t,D=1<<31,L={}.hasOwnPro
 perty,H=[],q=H.pop,_=H.push,M=H.push,O=H.slice,F=H.indexOf||function(e){var t=0,n=this.length;for(;n>t;t++)if(this[t]===e)return t;return-1},B="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",P="[\\x20\\t\\r\\n\\f]",R="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",W=R.replace("w","w#"),$="\\["+P+"*("+R+")"+P+"*(?:([*^$|!~]?=)"+P+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+W+")|)|)"+P+"*\\]",I=":("+R+")(?:\\(((['\"])((?:\\\\.|[^\\\\])*?)\\3|((?:\\\\.|[^\\\\()[\\]]|"+$.replace(3,8)+")*)|.*)\\)|)",z=RegExp("^"+P+"+|((?:^|[^\\\\])(?:\\\\.)*)"+P+"+$","g"),X=RegExp("^"+P+"*,"+P+"*"),U=RegExp("^"+P+"*([>+~]|"+P+")"+P+"*"),V=RegExp(P+"*[+~]"),Y=RegExp("="+P+"*([^\\]'\"]*)"+P+"*\\]","g"),J=RegExp(I),G=RegExp("^"+W+"$"),Q={ID:RegExp("^#("+R+")"),CLASS:RegExp("^\\.("+R+")"),TAG:RegExp("^("+R.replace("w","w*")+")"),ATTR:RegExp("^"+$),PSEUDO:RegExp("^"+I),CHILD:RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+P+"*(even|od
 d|(([+-]|)(\\d*)n|)"+P+"*(?:([+-]|)"+P+"*(\\d+)|))"+P+"*\\)|)","i"),bool:RegExp("^(?:"+B+")$","i"),needsContext:RegExp("^"+P+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+P+"*((?:-\\d)?\\d*)"+P+"*\\)|)(?=[^-]|$)","i")},K=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,et=/^(?:input|select|textarea|button)$/i,tt=/^h\d$/i,nt=/'|\\/g,rt=RegExp("\\\\([\\da-f]{1,6}"+P+"?|("+P+")|.)","ig"),it=function(e,t,n){var r="0x"+t-65536;return r!==r||n?t:0>r?String.fromCharCode(r+65536):String.fromCharCode(55296|r>>10,56320|1023&r)};try{M.apply(H=O.call(w.childNodes),w.childNodes),H[w.childNodes.length].nodeType}catch(ot){M={apply:H.length?function(e,t){_.apply(e,O.call(t))}:function(e,t){var n=e.length,r=0;while(e[n++]=t[r++]);e.length=n-1}}}function at(e,t,n,i){var o,a,s,l,u,c,d,m,y,x;if((t?t.ownerDocument||t:w)!==f&&p(t),t=t||f,n=n||[],!e||"string"!=typeof e)return n;if(1!==(l=t.nodeType)&&9!==l)return[];if(h&&!i){if(o=Z.exec(e))if(s=o[1]){if(9===l){if(a=t.getElementById(s),!a
 ||!a.parentNode)return n;if(a.id===s)return n.push(a),n}else if(t.ownerDocument&&(a=t.ownerDocument.getElementById(s))&&v(t,a)&&a.id===s)return n.push(a),n}else{if(o[2])return M.apply(n,t.getElementsByTagName(e)),n;if((s=o[3])&&r.getElementsByClassName&&t.getElementsByClassName)return M.apply(n,t.getElementsByClassName(s)),n}if(r.qsa&&(!g||!g.test(e))){if(m=d=b,y=t,x=9===l&&e,1===l&&"object"!==t.nodeName.toLowerCase()){c=bt(e),(d=t.getAttribute("id"))?m=d.replace(nt,"\\$&"):t.setAttribute("id",m),m="[id='"+m+"'] ",u=c.length;while(u--)c[u]=m+xt(c[u]);y=V.test(e)&&t.parentNode||t,x=c.join(",")}if(x)try{return M.apply(n,y.querySelectorAll(x)),n}catch(T){}finally{d||t.removeAttribute("id")}}}return At(e.replace(z,"$1"),t,n,i)}function st(e){return K.test(e+"")}function lt(){var e=[];function t(n,r){return e.push(n+=" ")>o.cacheLength&&delete t[e.shift()],t[n]=r}return t}function ut(e){return e[b]=!0,e}function ct(e){var t=f.createElement("div");try{return!!e(t)}catch(n){return!1}finall
 y{t.parentNode&&t.parentNode.removeChild(t),t=null}}function pt(e,t,n){e=e.split("|");var r,i=e.length,a=n?null:t;while(i--)(r=o.attrHandle[e[i]])&&r!==t||(o.attrHandle[e[i]]=a)}function ft(e,t){var n=e.getAttributeNode(t);return n&&n.specified?n.value:e[t]===!0?t.toLowerCase():null}function dt(e,t){return e.getAttribute(t,"type"===t.toLowerCase()?1:2)}function ht(e){return"input"===e.nodeName.toLowerCase()?e.defaultValue:t}function gt(e,t){var n=t&&e,r=n&&1===e.nodeType&&1===t.nodeType&&(~t.sourceIndex||D)-(~e.sourceIndex||D);if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function mt(e){return function(t){var n=t.nodeName.toLowerCase();return"input"===n&&t.type===e}}function yt(e){return function(t){var n=t.nodeName.toLowerCase();return("input"===n||"button"===n)&&t.type===e}}function vt(e){return ut(function(t){return t=+t,ut(function(n,r){var i,o=e([],n.length,t),a=o.length;while(a--)n[i=o[a]]&&(n[i]=!(r[i]=n[i]))})})}s=at.isXML=function(e){var t=e&&(e.o
 wnerDocument||e).documentElement;return t?"HTML"!==t.nodeName:!1},r=at.support={},p=at.setDocument=function(e){var n=e?e.ownerDocument||e:w,i=n.parentWindow;return n!==f&&9===n.nodeType&&n.documentElement?(f=n,d=n.documentElement,h=!s(n),i&&i.frameElement&&i.attachEvent("onbeforeunload",function(){p()}),r.attributes=ct(function(e){return e.innerHTML="<a href='#'></a>",pt("type|href|height|width",dt,"#"===e.firstChild.getAttribute("href")),pt(B,ft,null==e.getAttribute("disabled")),e.className="i",!e.getAttribute("className")}),r.input=ct(function(e){return e.innerHTML="<input>",e.firstChild.setAttribute("value",""),""===e.firstChild.getAttribute("value")}),pt("value",ht,r.attributes&&r.input),r.getElementsByTagName=ct(function(e){return e.appendChild(n.createComment("")),!e.getElementsByTagName("*").length}),r.getElementsByClassName=ct(function(e){return e.innerHTML="<div class='a'></div><div class='a i'></div>",e.firstChild.className="i",2===e.getElementsByClassName("i").length}),r.
 getById=ct(function(e){return d.appendChild(e).id=b,!n.getElementsByName||!n.getElementsByName(b).length}),r.getById?(o.find.ID=function(e,t){if(typeof t.getElementById!==j&&h){var n=t.getElementById(e);return n&&n.parentNode?[n]:[]}},o.filter.ID=function(e){var t=e.replace(rt,it);return function(e){return e.getAttribute("id")===t}}):(delete o.find.ID,o.filter.ID=function(e){var t=e.replace(rt,it);return function(e){var n=typeof e.getAttributeNode!==j&&e.getAttributeNode("id");return n&&n.value===t}}),o.find.TAG=r.getElementsByTagName?function(e,n){return typeof n.getElementsByTagName!==j?n.getElementsByTagName(e):t}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},o.find.CLASS=r.getElementsByClassName&&function(e,n){return typeof n.getElementsByClassName!==j&&h?n.getElementsByClassName(e):t},m=[],g=[],(r.qsa=st(n.querySelectorAll))&&(ct(function(e){e.innerHTML="<select><option selected=''></option></sel
 ect>",e.querySelectorAll("[selected]").length||g.push("\\["+P+"*(?:value|"+B+")"),e.querySelectorAll(":checked").length||g.push(":checked")}),ct(function(e){var t=n.createElement("input");t.setAttribute("type","hidden"),e.appendChild(t).setAttribute("t",""),e.querySelectorAll("[t^='']").length&&g.push("[*^$]="+P+"*(?:''|\"\")"),e.querySelectorAll(":enabled").length||g.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),g.push(",.*:")})),(r.matchesSelector=st(y=d.webkitMatchesSelector||d.mozMatchesSelector||d.oMatchesSelector||d.msMatchesSelector))&&ct(function(e){r.disconnectedMatch=y.call(e,"div"),y.call(e,"[s!='']:x"),m.push("!=",I)}),g=g.length&&RegExp(g.join("|")),m=m.length&&RegExp(m.join("|")),v=st(d.contains)||d.compareDocumentPosition?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if
 (t===e)return!0;return!1},r.sortDetached=ct(function(e){return 1&e.compareDocumentPosition(n.createElement("div"))}),A=d.compareDocumentPosition?function(e,t){if(e===t)return S=!0,0;var i=t.compareDocumentPosition&&e.compareDocumentPosition&&e.compareDocumentPosition(t);return i?1&i||!r.sortDetached&&t.compareDocumentPosition(e)===i?e===n||v(w,e)?-1:t===n||v(w,t)?1:c?F.call(c,e)-F.call(c,t):0:4&i?-1:1:e.compareDocumentPosition?-1:1}:function(e,t){var r,i=0,o=e.parentNode,a=t.parentNode,s=[e],l=[t];if(e===t)return S=!0,0;if(!o||!a)return e===n?-1:t===n?1:o?-1:a?1:c?F.call(c,e)-F.call(c,t):0;if(o===a)return gt(e,t);r=e;while(r=r.parentNode)s.unshift(r);r=t;while(r=r.parentNode)l.unshift(r);while(s[i]===l[i])i++;return i?gt(s[i],l[i]):s[i]===w?-1:l[i]===w?1:0},n):f},at.matches=function(e,t){return at(e,null,null,t)},at.matchesSelector=function(e,t){if((e.ownerDocument||e)!==f&&p(e),t=t.replace(Y,"='$1']"),!(!r.matchesSelector||!h||m&&m.test(t)||g&&g.test(t)))try{var n=y.call(e,t);if(n|
 |r.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(i){}return at(t,f,null,[e]).length>0},at.contains=function(e,t){return(e.ownerDocument||e)!==f&&p(e),v(e,t)},at.attr=function(e,n){(e.ownerDocument||e)!==f&&p(e);var i=o.attrHandle[n.toLowerCase()],a=i&&L.call(o.attrHandle,n.toLowerCase())?i(e,n,!h):t;return a===t?r.attributes||!h?e.getAttribute(n):(a=e.getAttributeNode(n))&&a.specified?a.value:null:a},at.error=function(e){throw Error("Syntax error, unrecognized expression: "+e)},at.uniqueSort=function(e){var t,n=[],i=0,o=0;if(S=!r.detectDuplicates,c=!r.sortStable&&e.slice(0),e.sort(A),S){while(t=e[o++])t===e[o]&&(i=n.push(o));while(i--)e.splice(n[i],1)}return e},a=at.getText=function(e){var t,n="",r=0,i=e.nodeType;if(i){if(1===i||9===i||11===i){if("string"==typeof e.textContent)return e.textContent;for(e=e.firstChild;e;e=e.nextSibling)n+=a(e)}else if(3===i||4===i)return e.nodeValue}else for(;t=e[r];r++)n+=a(t);return n},o=at.selectors={cacheLength:50,createPs
 eudo:ut,match:Q,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(rt,it),e[3]=(e[4]||e[5]||"").replace(rt,it),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||at.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&at.error(e[0]),e},PSEUDO:function(e){var n,r=!e[5]&&e[2];return Q.CHILD.test(e[0])?null:(e[3]&&e[4]!==t?e[2]=e[4]:r&&J.test(r)&&(n=bt(r,!0))&&(n=r.indexOf(")",r.length-n)-r.length)&&(e[0]=e[0].slice(0,n),e[2]=r.slice(0,n)),e.slice(0,3))}},filter:{TAG:function(e){var t=e.replace(rt,it).toLowerCase();return"*"===e?function(){return!0}:function(e){return e.nodeName&&e.nodeName.toLowerCase()===t}},CLASS:function(e){var t=N[e+" "];return t||(t=RegExp("(^|"+P+")"+e+"("+P+"|$)"))&&N(e,functio
 n(e){return t.test("string"==typeof e.className&&e.className||typeof e.getAttribute!==j&&e.getAttribute("class")||"")})},ATTR:function(e,t,n){return function(r){var i=at.attr(r,e);return null==i?"!="===t:t?(i+="","="===t?i===n:"!="===t?i!==n:"^="===t?n&&0===i.indexOf(n):"*="===t?n&&i.indexOf(n)>-1:"$="===t?n&&i.slice(-n.length)===n:"~="===t?(" "+i+" ").indexOf(n)>-1:"|="===t?i===n||i.slice(0,n.length+1)===n+"-":!1):!0}},CHILD:function(e,t,n,r,i){var o="nth"!==e.slice(0,3),a="last"!==e.slice(-4),s="of-type"===t;return 1===r&&0===i?function(e){return!!e.parentNode}:function(t,n,l){var u,c,p,f,d,h,g=o!==a?"nextSibling":"previousSibling",m=t.parentNode,y=s&&t.nodeName.toLowerCase(),v=!l&&!s;if(m){if(o){while(g){p=t;while(p=p[g])if(s?p.nodeName.toLowerCase()===y:1===p.nodeType)return!1;h=g="only"===e&&!h&&"nextSibling"}return!0}if(h=[a?m.firstChild:m.lastChild],a&&v){c=m[b]||(m[b]={}),u=c[e]||[],d=u[0]===T&&u[1],f=u[0]===T&&u[2],p=d&&m.childNodes[d];while(p=++d&&p&&p[g]||(f=d=0)||h.pop()
 )if(1===p.nodeType&&++f&&p===t){c[e]=[T,d,f];break}}else if(v&&(u=(t[b]||(t[b]={}))[e])&&u[0]===T)f=u[1];else while(p=++d&&p&&p[g]||(f=d=0)||h.pop())if((s?p.nodeName.toLowerCase()===y:1===p.nodeType)&&++f&&(v&&((p[b]||(p[b]={}))[e]=[T,f]),p===t))break;return f-=i,f===r||0===f%r&&f/r>=0}}},PSEUDO:function(e,t){var n,r=o.pseudos[e]||o.setFilters[e.toLowerCase()]||at.error("unsupported pseudo: "+e);return r[b]?r(t):r.length>1?(n=[e,e,"",t],o.setFilters.hasOwnProperty(e.toLowerCase())?ut(function(e,n){var i,o=r(e,t),a=o.length;while(a--)i=F.call(e,o[a]),e[i]=!(n[i]=o[a])}):function(e){return r(e,0,n)}):r}},pseudos:{not:ut(function(e){var t=[],n=[],r=l(e.replace(z,"$1"));return r[b]?ut(function(e,t,n,i){var o,a=r(e,null,i,[]),s=e.length;while(s--)(o=a[s])&&(e[s]=!(t[s]=o))}):function(e,i,o){return t[0]=e,r(t,null,o,n),!n.pop()}}),has:ut(function(e){return function(t){return at(e,t).length>0}}),contains:ut(function(e){return function(t){return(t.textContent||t.innerText||a(t)).indexOf(e)>
 -1}}),lang:ut(function(e){return G.test(e||"")||at.error("unsupported lang: "+e),e=e.replace(rt,it).toLowerCase(),function(t){var n;do if(n=h?t.lang:t.getAttribute("xml:lang")||t.getAttribute("lang"))return n=n.toLowerCase(),n===e||0===n.indexOf(e+"-");while((t=t.parentNode)&&1===t.nodeType);return!1}}),target:function(t){var n=e.location&&e.location.hash;return n&&n.slice(1)===t.id},root:function(e){return e===d},focus:function(e){return e===f.activeElement&&(!f.hasFocus||f.hasFocus())&&!!(e.type||e.href||~e.tabIndex)},enabled:function(e){return e.disabled===!1},disabled:function(e){return e.disabled===!0},checked:function(e){var t=e.nodeName.toLowerCase();return"input"===t&&!!e.checked||"option"===t&&!!e.selected},selected:function(e){return e.parentNode&&e.parentNode.selectedIndex,e.selected===!0},empty:function(e){for(e=e.firstChild;e;e=e.nextSibling)if(e.nodeName>"@"||3===e.nodeType||4===e.nodeType)return!1;return!0},parent:function(e){return!o.pseudos.empty(e)},header:function
 (e){return tt.test(e.nodeName)},input:function(e){return et.test(e.nodeName)},button:function(e){var t=e.nodeName.toLowerCase();return"input"===t&&"button"===e.type||"button"===t},text:function(e){var t;return"input"===e.nodeName.toLowerCase()&&"text"===e.type&&(null==(t=e.getAttribute("type"))||t.toLowerCase()===e.type)},first:vt(function(){return[0]}),last:vt(function(e,t){return[t-1]}),eq:vt(function(e,t,n){return[0>n?n+t:n]}),even:vt(function(e,t){var n=0;for(;t>n;n+=2)e.push(n);return e}),odd:vt(function(e,t){var n=1;for(;t>n;n+=2)e.push(n);return e}),lt:vt(function(e,t,n){var r=0>n?n+t:n;for(;--r>=0;)e.push(r);return e}),gt:vt(function(e,t,n){var r=0>n?n+t:n;for(;t>++r;)e.push(r);return e})}};for(n in{radio:!0,checkbox:!0,file:!0,password:!0,image:!0})o.pseudos[n]=mt(n);for(n in{submit:!0,reset:!0})o.pseudos[n]=yt(n);function bt(e,t){var n,r,i,a,s,l,u,c=k[e+" "];if(c)return t?0:c.slice(0);s=e,l=[],u=o.preFilter;while(s){(!n||(r=X.exec(s)))&&(r&&(s=s.slice(r[0].length)||s),l.pu
 sh(i=[])),n=!1,(r=U.exec(s))&&(n=r.shift(),i.push({value:n,type:r[0].replace(z," ")}),s=s.slice(n.length));for(a in o.filter)!(r=Q[a].exec(s))||u[a]&&!(r=u[a](r))||(n=r.shift(),i.push({value:n,type:a,matches:r}),s=s.slice(n.length));if(!n)break}return t?s.length:s?at.error(e):k(e,l).slice(0)}function xt(e){var t=0,n=e.length,r="";for(;n>t;t++)r+=e[t].value;return r}function wt(e,t,n){var r=t.dir,o=n&&"parentNode"===r,a=C++;return t.first?function(t,n,i){while(t=t[r])if(1===t.nodeType||o)return e(t,n,i)}:function(t,n,s){var l,u,c,p=T+" "+a;if(s){while(t=t[r])if((1===t.nodeType||o)&&e(t,n,s))return!0}else while(t=t[r])if(1===t.nodeType||o)if(c=t[b]||(t[b]={}),(u=c[r])&&u[0]===p){if((l=u[1])===!0||l===i)return l===!0}else if(u=c[r]=[p],u[1]=e(t,n,s)||i,u[1]===!0)return!0}}function Tt(e){return e.length>1?function(t,n,r){var i=e.length;while(i--)if(!e[i](t,n,r))return!1;return!0}:e[0]}function Ct(e,t,n,r,i){var o,a=[],s=0,l=e.length,u=null!=t;for(;l>s;s++)(o=e[s])&&(!n||n(o,r,i))&&(a.pu
 sh(o),u&&t.push(s));return a}function Nt(e,t,n,r,i,o){return r&&!r[b]&&(r=Nt(r)),i&&!i[b]&&(i=Nt(i,o)),ut(function(o,a,s,l){var u,c,p,f=[],d=[],h=a.length,g=o||St(t||"*",s.nodeType?[s]:s,[]),m=!e||!o&&t?g:Ct(g,f,e,s,l),y=n?i||(o?e:h||r)?[]:a:m;if(n&&n(m,y,s,l),r){u=Ct(y,d),r(u,[],s,l),c=u.length;while(c--)(p=u[c])&&(y[d[c]]=!(m[d[c]]=p))}if(o){if(i||e){if(i){u=[],c=y.length;while(c--)(p=y[c])&&u.push(m[c]=p);i(null,y=[],u,l)}c=y.length;while(c--)(p=y[c])&&(u=i?F.call(o,p):f[c])>-1&&(o[u]=!(a[u]=p))}}else y=Ct(y===a?y.splice(h,y.length):y),i?i(null,a,y,l):M.apply(a,y)})}function kt(e){var t,n,r,i=e.length,a=o.relative[e[0].type],s=a||o.relative[" "],l=a?1:0,c=wt(function(e){return e===t},s,!0),p=wt(function(e){return F.call(t,e)>-1},s,!0),f=[function(e,n,r){return!a&&(r||n!==u)||((t=n).nodeType?c(e,n,r):p(e,n,r))}];for(;i>l;l++)if(n=o.relative[e[l].type])f=[wt(Tt(f),n)];else{if(n=o.filter[e[l].type].apply(null,e[l].matches),n[b]){for(r=++l;i>r;r++)if(o.relative[e[r].type])break;retur
 n Nt(l>1&&Tt(f),l>1&&xt(e.slice(0,l-1).concat({value:" "===e[l-2].type?"*":""})).replace(z,"$1"),n,r>l&&kt(e.slice(l,r)),i>r&&kt(e=e.slice(r)),i>r&&xt(e))}f.push(n)}return Tt(f)}function Et(e,t){var n=0,r=t.length>0,a=e.length>0,s=function(s,l,c,p,d){var h,g,m,y=[],v=0,b="0",x=s&&[],w=null!=d,C=u,N=s||a&&o.find.TAG("*",d&&l.parentNode||l),k=T+=null==C?1:Math.random()||.1;for(w&&(u=l!==f&&l,i=n);null!=(h=N[b]);b++){if(a&&h){g=0;while(m=e[g++])if(m(h,l,c)){p.push(h);break}w&&(T=k,i=++n)}r&&((h=!m&&h)&&v--,s&&x.push(h))}if(v+=b,r&&b!==v){g=0;while(m=t[g++])m(x,y,l,c);if(s){if(v>0)while(b--)x[b]||y[b]||(y[b]=q.call(p));y=Ct(y)}M.apply(p,y),w&&!s&&y.length>0&&v+t.length>1&&at.uniqueSort(p)}return w&&(T=k,u=C),x};return r?ut(s):s}l=at.compile=function(e,t){var n,r=[],i=[],o=E[e+" "];if(!o){t||(t=bt(e)),n=t.length;while(n--)o=kt(t[n]),o[b]?r.push(o):i.push(o);o=E(e,Et(i,r))}return o};function St(e,t,n){var r=0,i=t.length;for(;i>r;r++)at(e,t[r],n);return n}function At(e,t,n,i){var a,s,u,c,p
 ,f=bt(e);if(!i&&1===f.length){if(s=f[0]=f[0].slice(0),s.length>2&&"ID"===(u=s[0]).type&&r.getById&&9===t.nodeType&&h&&o.relative[s[1].type]){if(t=(o.find.ID(u.matches[0].replace(rt,it),t)||[])[0],!t)return n;e=e.slice(s.shift().value.length)}a=Q.needsContext.test(e)?0:s.length;while(a--){if(u=s[a],o.relative[c=u.type])break;if((p=o.find[c])&&(i=p(u.matches[0].replace(rt,it),V.test(s[0].type)&&t.parentNode||t))){if(s.splice(a,1),e=i.length&&xt(s),!e)return M.apply(n,i),n;break}}}return l(e,f)(i,t,!h,n,V.test(e)),n}o.pseudos.nth=o.pseudos.eq;function jt(){}jt.prototype=o.filters=o.pseudos,o.setFilters=new jt,r.sortStable=b.split("").sort(A).join("")===b,p(),[0,0].sort(A),r.detectDuplicates=S,x.find=at,x.expr=at.selectors,x.expr[":"]=x.expr.pseudos,x.unique=at.uniqueSort,x.text=at.getText,x.isXMLDoc=at.isXML,x.contains=at.contains}(e);var O={};function F(e){var t=O[e]={};return x.each(e.match(T)||[],function(e,n){t[n]=!0}),t}x.Callbacks=function(e){e="string"==typeof e?O[e]||F(e):x.ext
 end({},e);var n,r,i,o,a,s,l=[],u=!e.once&&[],c=function(t){for(r=e.memory&&t,i=!0,a=s||0,s=0,o=l.length,n=!0;l&&o>a;a++)if(l[a].apply(t[0],t[1])===!1&&e.stopOnFalse){r=!1;break}n=!1,l&&(u?u.length&&c(u.shift()):r?l=[]:p.disable())},p={add:function(){if(l){var t=l.length;(function i(t){x.each(t,function(t,n){var r=x.type(n);"function"===r?e.unique&&p.has(n)||l.push(n):n&&n.length&&"string"!==r&&i(n)})})(arguments),n?o=l.length:r&&(s=t,c(r))}return this},remove:function(){return l&&x.each(arguments,function(e,t){var r;while((r=x.inArray(t,l,r))>-1)l.splice(r,1),n&&(o>=r&&o--,a>=r&&a--)}),this},has:function(e){return e?x.inArray(e,l)>-1:!(!l||!l.length)},empty:function(){return l=[],o=0,this},disable:function(){return l=u=r=t,this},disabled:function(){return!l},lock:function(){return u=t,r||p.disable(),this},locked:function(){return!u},fireWith:function(e,t){return t=t||[],t=[e,t.slice?t.slice():t],!l||i&&!u||(n?u.push(t):c(t)),this},fire:function(){return p.fireWith(this,arguments),th
 is},fired:function(){return!!i}};return p},x.extend({Deferred:function(e){var t=[["resolve","done",x.Callbacks("once memory"),"resolved"],["reject","fail",x.Callbacks("once memory"),"rejected"],["notify","progress",x.Callbacks("memory")]],n="pending",r={state:function(){return n},always:function(){return i.done(arguments).fail(arguments),this},then:function(){var e=arguments;return x.Deferred(function(n){x.each(t,function(t,o){var a=o[0],s=x.isFunction(e[t])&&e[t];i[o[1]](function(){var e=s&&s.apply(this,arguments);e&&x.isFunction(e.promise)?e.promise().done(n.resolve).fail(n.reject).progress(n.notify):n[a+"With"](this===r?n.promise():this,s?[e]:arguments)})}),e=null}).promise()},promise:function(e){return null!=e?x.extend(e,r):r}},i={};return r.pipe=r.then,x.each(t,function(e,o){var a=o[2],s=o[3];r[o[1]]=a.add,s&&a.add(function(){n=s},t[1^e][2].disable,t[2][2].lock),i[o[0]]=function(){return i[o[0]+"With"](this===i?r:this,arguments),this},i[o[0]+"With"]=a.fireWith}),r.promise(i),e&
 &e.call(i,i),i},when:function(e){var t=0,n=g.call(arguments),r=n.length,i=1!==r||e&&x.isFunction(e.promise)?r:0,o=1===i?e:x.Deferred(),a=function(e,t,n){return function(r){t[e]=this,n[e]=arguments.length>1?g.call(arguments):r,n===s?o.notifyWith(t,n):--i||o.resolveWith(t,n)}},s,l,u;if(r>1)for(s=Array(r),l=Array(r),u=Array(r);r>t;t++)n[t]&&x.isFunction(n[t].promise)?n[t].promise().done(a(t,u,n)).fail(o.reject).progress(a(t,l,s)):--i;return i||o.resolveWith(u,n),o.promise()}}),x.support=function(t){var n,r,o,s,l,u,c,p,f,d=a.createElement("div");if(d.setAttribute("className","t"),d.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",n=d.getElementsByTagName("*")||[],r=d.getElementsByTagName("a")[0],!r||!r.style||!n.length)return t;s=a.createElement("select"),u=s.appendChild(a.createElement("option")),o=d.getElementsByTagName("input")[0],r.style.cssText="top:1px;float:left;opacity:.5",t.getSetAttribute="t"!==d.className,t.leadingWhitespace=3===d.firstChild.node
 Type,t.tbody=!d.getElementsByTagName("tbody").length,t.htmlSerialize=!!d.getElementsByTagName("link").length,t.style=/top/.test(r.getAttribute("style")),t.hrefNormalized="/a"===r.getAttribute("href"),t.opacity=/^0.5/.test(r.style.opacity),t.cssFloat=!!r.style.cssFloat,t.checkOn=!!o.value,t.optSelected=u.selected,t.enctype=!!a.createElement("form").enctype,t.html5Clone="<:nav></:nav>"!==a.createElement("nav").cloneNode(!0).outerHTML,t.inlineBlockNeedsLayout=!1,t.shrinkWrapBlocks=!1,t.pixelPosition=!1,t.deleteExpando=!0,t.noCloneEvent=!0,t.reliableMarginRight=!0,t.boxSizingReliable=!0,o.checked=!0,t.noCloneChecked=o.cloneNode(!0).checked,s.disabled=!0,t.optDisabled=!u.disabled;try{delete d.test}catch(h){t.deleteExpando=!1}o=a.createElement("input"),o.setAttribute("value",""),t.input=""===o.getAttribute("value"),o.value="t",o.setAttribute("type","radio"),t.radioValue="t"===o.value,o.setAttribute("checked","t"),o.setAttribute("name","t"),l=a.createDocumentFragment(),l.appendChild(o),t.a
 ppendChecked=o.checked,t.checkClone=l.cloneNode(!0).cloneNode(!0).lastChild.checked,d.attachEvent&&(d.attachEvent("onclick",function(){t.noCloneEvent=!1}),d.cloneNode(!0).click());for(f in{submit:!0,change:!0,focusin:!0})d.setAttribute(c="on"+f,"t"),t[f+"Bubbles"]=c in e||d.attributes[c].expando===!1;d.style.backgroundClip="content-box",d.cloneNode(!0).style.backgroundClip="",t.clearCloneStyle="content-box"===d.style.backgroundClip;for(f in x(t))break;return t.ownLast="0"!==f,x(function(){var n,r,o,s="padding:0;margin:0;border:0;display:block;box-sizing:content-box;-moz-box-sizing:content-box;-webkit-box-sizing:content-box;",l=a.getElementsByTagName("body")[0];l&&(n=a.createElement("div"),n.style.cssText="border:0;width:0;height:0;position:absolute;top:0;left:-9999px;margin-top:1px",l.appendChild(n).appendChild(d),d.innerHTML="<table><tr><td></td><td>t</td></tr></table>",o=d.getElementsByTagName("td"),o[0].style.cssText="padding:0;margin:0;border:0;display:none",p=0===o[0].offsetHei
 ght,o[0].style.display="",o[1].style.display="none",t.reliableHiddenOffsets=p&&0===o[0].offsetHeight,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-top:1%;position:absolute;top:1%;",x.swap(l,null!=l.style.zoom?{zoom:1}:{},function(){t.boxSizing=4===d.offsetWidth}),e.getComputedStyle&&(t.pixelPosition="1%"!==(e.getComputedStyle(d,null)||{}).top,t.boxSizingReliable="4px"===(e.getComputedStyle(d,null)||{width:"4px"}).width,r=d.appendChild(a.createElement("div")),r.style.cssText=d.style.cssText=s,r.style.marginRight=r.style.width="0",d.style.width="1px",t.reliableMarginRight=!parseFloat((e.getComputedStyle(r,null)||{}).marginRight)),typeof d.style.zoom!==i&&(d.innerHTML="",d.style.cssText=s+"width:1px;padding:1px;display:inline;zoom:1",t.inlineBlockNeedsLayout=3===d.offsetWidth,d.style.display="block",d.innerHTML="<div></div>",d.firstChild.style.width="5px",t.shrinkWrapB
 locks=3!==d.offsetWidth,t.inlineBlockNeedsLayout&&(l.style.zoom=1)),l.removeChild(n),n=d=o=r=null)
-}),n=s=l=u=r=o=null,t}({});var B=/(?:\{[\s\S]*\}|\[[\s\S]*\])$/,P=/([A-Z])/g;function R(e,n,r,i){if(x.acceptData(e)){var o,a,s=x.expando,l=e.nodeType,u=l?x.cache:e,c=l?e[s]:e[s]&&s;if(c&&u[c]&&(i||u[c].data)||r!==t||"string"!=typeof n)return c||(c=l?e[s]=p.pop()||x.guid++:s),u[c]||(u[c]=l?{}:{toJSON:x.noop}),("object"==typeof n||"function"==typeof n)&&(i?u[c]=x.extend(u[c],n):u[c].data=x.extend(u[c].data,n)),a=u[c],i||(a.data||(a.data={}),a=a.data),r!==t&&(a[x.camelCase(n)]=r),"string"==typeof n?(o=a[n],null==o&&(o=a[x.camelCase(n)])):o=a,o}}function W(e,t,n){if(x.acceptData(e)){var r,i,o=e.nodeType,a=o?x.cache:e,s=o?e[x.expando]:x.expando;if(a[s]){if(t&&(r=n?a[s]:a[s].data)){x.isArray(t)?t=t.concat(x.map(t,x.camelCase)):t in r?t=[t]:(t=x.camelCase(t),t=t in r?[t]:t.split(" ")),i=t.length;while(i--)delete r[t[i]];if(n?!I(r):!x.isEmptyObject(r))return}(n||(delete a[s].data,I(a[s])))&&(o?x.cleanData([e],!0):x.support.deleteExpando||a!=a.window?delete a[s]:a[s]=null)}}}x.extend({cache:
 {},noData:{applet:!0,embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(e){return e=e.nodeType?x.cache[e[x.expando]]:e[x.expando],!!e&&!I(e)},data:function(e,t,n){return R(e,t,n)},removeData:function(e,t){return W(e,t)},_data:function(e,t,n){return R(e,t,n,!0)},_removeData:function(e,t){return W(e,t,!0)},acceptData:function(e){if(e.nodeType&&1!==e.nodeType&&9!==e.nodeType)return!1;var t=e.nodeName&&x.noData[e.nodeName.toLowerCase()];return!t||t!==!0&&e.getAttribute("classid")===t}}),x.fn.extend({data:function(e,n){var r,i,o=null,a=0,s=this[0];if(e===t){if(this.length&&(o=x.data(s),1===s.nodeType&&!x._data(s,"parsedAttrs"))){for(r=s.attributes;r.length>a;a++)i=r[a].name,0===i.indexOf("data-")&&(i=x.camelCase(i.slice(5)),$(s,i,o[i]));x._data(s,"parsedAttrs",!0)}return o}return"object"==typeof e?this.each(function(){x.data(this,e)}):arguments.length>1?this.each(function(){x.data(this,e,n)}):s?$(s,e,x.data(s,e)):null},removeData:function(e){return this.each(f
 unction(){x.removeData(this,e)})}});function $(e,n,r){if(r===t&&1===e.nodeType){var i="data-"+n.replace(P,"-$1").toLowerCase();if(r=e.getAttribute(i),"string"==typeof r){try{r="true"===r?!0:"false"===r?!1:"null"===r?null:+r+""===r?+r:B.test(r)?x.parseJSON(r):r}catch(o){}x.data(e,n,r)}else r=t}return r}function I(e){var t;for(t in e)if(("data"!==t||!x.isEmptyObject(e[t]))&&"toJSON"!==t)return!1;return!0}x.extend({queue:function(e,n,r){var i;return e?(n=(n||"fx")+"queue",i=x._data(e,n),r&&(!i||x.isArray(r)?i=x._data(e,n,x.makeArray(r)):i.push(r)),i||[]):t},dequeue:function(e,t){t=t||"fx";var n=x.queue(e,t),r=n.length,i=n.shift(),o=x._queueHooks(e,t),a=function(){x.dequeue(e,t)};"inprogress"===i&&(i=n.shift(),r--),i&&("fx"===t&&n.unshift("inprogress"),delete o.stop,i.call(e,a,o)),!r&&o&&o.empty.fire()},_queueHooks:function(e,t){var n=t+"queueHooks";return x._data(e,n)||x._data(e,n,{empty:x.Callbacks("once memory").add(function(){x._removeData(e,t+"queue"),x._removeData(e,n)})})}}),x.fn
 .extend({queue:function(e,n){var r=2;return"string"!=typeof e&&(n=e,e="fx",r--),r>arguments.length?x.queue(this[0],e):n===t?this:this.each(function(){var t=x.queue(this,e,n);x._queueHooks(this,e),"fx"===e&&"inprogress"!==t[0]&&x.dequeue(this,e)})},dequeue:function(e){return this.each(function(){x.dequeue(this,e)})},delay:function(e,t){return e=x.fx?x.fx.speeds[e]||e:e,t=t||"fx",this.queue(t,function(t,n){var r=setTimeout(t,e);n.stop=function(){clearTimeout(r)}})},clearQueue:function(e){return this.queue(e||"fx",[])},promise:function(e,n){var r,i=1,o=x.Deferred(),a=this,s=this.length,l=function(){--i||o.resolveWith(a,[a])};"string"!=typeof e&&(n=e,e=t),e=e||"fx";while(s--)r=x._data(a[s],e+"queueHooks"),r&&r.empty&&(i++,r.empty.add(l));return l(),o.promise(n)}});var z,X,U=/[\t\r\n\f]/g,V=/\r/g,Y=/^(?:input|select|textarea|button|object)$/i,J=/^(?:a|area)$/i,G=/^(?:checked|selected)$/i,Q=x.support.getSetAttribute,K=x.support.input;x.fn.extend({attr:function(e,t){return x.access(this,x.
 attr,e,t,arguments.length>1)},removeAttr:function(e){return this.each(function(){x.removeAttr(this,e)})},prop:function(e,t){return x.access(this,x.prop,e,t,arguments.length>1)},removeProp:function(e){return e=x.propFix[e]||e,this.each(function(){try{this[e]=t,delete this[e]}catch(n){}})},addClass:function(e){var t,n,r,i,o,a=0,s=this.length,l="string"==typeof e&&e;if(x.isFunction(e))return this.each(function(t){x(this).addClass(e.call(this,t,this.className))});if(l)for(t=(e||"").match(T)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(U," "):" ")){o=0;while(i=t[o++])0>r.indexOf(" "+i+" ")&&(r+=i+" ");n.className=x.trim(r)}return this},removeClass:function(e){var t,n,r,i,o,a=0,s=this.length,l=0===arguments.length||"string"==typeof e&&e;if(x.isFunction(e))return this.each(function(t){x(this).removeClass(e.call(this,t,this.className))});if(l)for(t=(e||"").match(T)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(
 U," "):"")){o=0;while(i=t[o++])while(r.indexOf(" "+i+" ")>=0)r=r.replace(" "+i+" "," ");n.className=e?x.trim(r):""}return this},toggleClass:function(e,t){var n=typeof e,r="boolean"==typeof t;return x.isFunction(e)?this.each(function(n){x(this).toggleClass(e.call(this,n,this.className,t),t)}):this.each(function(){if("string"===n){var o,a=0,s=x(this),l=t,u=e.match(T)||[];while(o=u[a++])l=r?l:!s.hasClass(o),s[l?"addClass":"removeClass"](o)}else(n===i||"boolean"===n)&&(this.className&&x._data(this,"__className__",this.className),this.className=this.className||e===!1?"":x._data(this,"__className__")||"")})},hasClass:function(e){var t=" "+e+" ",n=0,r=this.length;for(;r>n;n++)if(1===this[n].nodeType&&(" "+this[n].className+" ").replace(U," ").indexOf(t)>=0)return!0;return!1},val:function(e){var n,r,i,o=this[0];{if(arguments.length)return i=x.isFunction(e),this.each(function(n){var o;1===this.nodeType&&(o=i?e.call(this,n,x(this).val()):e,null==o?o="":"number"==typeof o?o+="":x.isArray(o)&&(
 o=x.map(o,function(e){return null==e?"":e+""})),r=x.valHooks[this.type]||x.valHooks[this.nodeName.toLowerCase()],r&&"set"in r&&r.set(this,o,"value")!==t||(this.value=o))});if(o)return r=x.valHooks[o.type]||x.valHooks[o.nodeName.toLowerCase()],r&&"get"in r&&(n=r.get(o,"value"))!==t?n:(n=o.value,"string"==typeof n?n.replace(V,""):null==n?"":n)}}}),x.extend({valHooks:{option:{get:function(e){var t=x.find.attr(e,"value");return null!=t?t:e.text}},select:{get:function(e){var t,n,r=e.options,i=e.selectedIndex,o="select-one"===e.type||0>i,a=o?null:[],s=o?i+1:r.length,l=0>i?s:o?i:0;for(;s>l;l++)if(n=r[l],!(!n.selected&&l!==i||(x.support.optDisabled?n.disabled:null!==n.getAttribute("disabled"))||n.parentNode.disabled&&x.nodeName(n.parentNode,"optgroup"))){if(t=x(n).val(),o)return t;a.push(t)}return a},set:function(e,t){var n,r,i=e.options,o=x.makeArray(t),a=i.length;while(a--)r=i[a],(r.selected=x.inArray(x(r).val(),o)>=0)&&(n=!0);return n||(e.selectedIndex=-1),o}}},attr:function(e,n,r){var o
 ,a,s=e.nodeType;if(e&&3!==s&&8!==s&&2!==s)return typeof e.getAttribute===i?x.prop(e,n,r):(1===s&&x.isXMLDoc(e)||(n=n.toLowerCase(),o=x.attrHooks[n]||(x.expr.match.bool.test(n)?X:z)),r===t?o&&"get"in o&&null!==(a=o.get(e,n))?a:(a=x.find.attr(e,n),null==a?t:a):null!==r?o&&"set"in o&&(a=o.set(e,r,n))!==t?a:(e.setAttribute(n,r+""),r):(x.removeAttr(e,n),t))},removeAttr:function(e,t){var n,r,i=0,o=t&&t.match(T);if(o&&1===e.nodeType)while(n=o[i++])r=x.propFix[n]||n,x.expr.match.bool.test(n)?K&&Q||!G.test(n)?e[r]=!1:e[x.camelCase("default-"+n)]=e[r]=!1:x.attr(e,n,""),e.removeAttribute(Q?n:r)},attrHooks:{type:{set:function(e,t){if(!x.support.radioValue&&"radio"===t&&x.nodeName(e,"input")){var n=e.value;return e.setAttribute("type",t),n&&(e.value=n),t}}}},propFix:{"for":"htmlFor","class":"className"},prop:function(e,n,r){var i,o,a,s=e.nodeType;if(e&&3!==s&&8!==s&&2!==s)return a=1!==s||!x.isXMLDoc(e),a&&(n=x.propFix[n]||n,o=x.propHooks[n]),r!==t?o&&"set"in o&&(i=o.set(e,r,n))!==t?i:e[n]=r:o&&"
 get"in o&&null!==(i=o.get(e,n))?i:e[n]},propHooks:{tabIndex:{get:function(e){var t=x.find.attr(e,"tabindex");return t?parseInt(t,10):Y.test(e.nodeName)||J.test(e.nodeName)&&e.href?0:-1}}}}),X={set:function(e,t,n){return t===!1?x.removeAttr(e,n):K&&Q||!G.test(n)?e.setAttribute(!Q&&x.propFix[n]||n,n):e[x.camelCase("default-"+n)]=e[n]=!0,n}},x.each(x.expr.match.bool.source.match(/\w+/g),function(e,n){var r=x.expr.attrHandle[n]||x.find.attr;x.expr.attrHandle[n]=K&&Q||!G.test(n)?function(e,n,i){var o=x.expr.attrHandle[n],a=i?t:(x.expr.attrHandle[n]=t)!=r(e,n,i)?n.toLowerCase():null;return x.expr.attrHandle[n]=o,a}:function(e,n,r){return r?t:e[x.camelCase("default-"+n)]?n.toLowerCase():null}}),K&&Q||(x.attrHooks.value={set:function(e,n,r){return x.nodeName(e,"input")?(e.defaultValue=n,t):z&&z.set(e,n,r)}}),Q||(z={set:function(e,n,r){var i=e.getAttributeNode(r);return i||e.setAttributeNode(i=e.ownerDocument.createAttribute(r)),i.value=n+="","value"===r||n===e.getAttribute(r)?n:t}},x.expr.a
 ttrHandle.id=x.expr.attrHandle.name=x.expr.attrHandle.coords=function(e,n,r){var i;return r?t:(i=e.getAttributeNode(n))&&""!==i.value?i.value:null},x.valHooks.button={get:function(e,n){var r=e.getAttributeNode(n);return r&&r.specified?r.value:t},set:z.set},x.attrHooks.contenteditable={set:function(e,t,n){z.set(e,""===t?!1:t,n)}},x.each(["width","height"],function(e,n){x.attrHooks[n]={set:function(e,r){return""===r?(e.setAttribute(n,"auto"),r):t}}})),x.support.hrefNormalized||x.each(["href","src"],function(e,t){x.propHooks[t]={get:function(e){return e.getAttribute(t,4)}}}),x.support.style||(x.attrHooks.style={get:function(e){return e.style.cssText||t},set:function(e,t){return e.style.cssText=t+""}}),x.support.optSelected||(x.propHooks.selected={get:function(e){var t=e.parentNode;return t&&(t.selectedIndex,t.parentNode&&t.parentNode.selectedIndex),null}}),x.each(["tabIndex","readOnly","maxLength","cellSpacing","cellPadding","rowSpan","colSpan","useMap","frameBorder","contentEditable"]
 ,function(){x.propFix[this.toLowerCase()]=this}),x.support.enctype||(x.propFix.enctype="encoding"),x.each(["radio","checkbox"],function(){x.valHooks[this]={set:function(e,n){return x.isArray(n)?e.checked=x.inArray(x(e).val(),n)>=0:t}},x.support.checkOn||(x.valHooks[this].get=function(e){return null===e.getAttribute("value")?"on":e.value})});var Z=/^(?:input|select|textarea)$/i,et=/^key/,tt=/^(?:mouse|contextmenu)|click/,nt=/^(?:focusinfocus|focusoutblur)$/,rt=/^([^.]*)(?:\.(.+)|)$/;function it(){return!0}function ot(){return!1}function at(){try{return a.activeElement}catch(e){}}x.event={global:{},add:function(e,n,r,o,a){var s,l,u,c,p,f,d,h,g,m,y,v=x._data(e);if(v){r.handler&&(c=r,r=c.handler,a=c.selector),r.guid||(r.guid=x.guid++),(l=v.events)||(l=v.events={}),(f=v.handle)||(f=v.handle=function(e){return typeof x===i||e&&x.event.triggered===e.type?t:x.event.dispatch.apply(f.elem,arguments)},f.elem=e),n=(n||"").match(T)||[""],u=n.length;while(u--)s=rt.exec(n[u])||[],g=y=s[1],m=(s[2]|
 |"").split(".").sort(),g&&(p=x.event.special[g]||{},g=(a?p.delegateType:p.bindType)||g,p=x.event.special[g]||{},d=x.extend({type:g,origType:y,data:o,handler:r,guid:r.guid,selector:a,needsContext:a&&x.expr.match.needsContext.test(a),namespace:m.join(".")},c),(h=l[g])||(h=l[g]=[],h.delegateCount=0,p.setup&&p.setup.call(e,o,m,f)!==!1||(e.addEventListener?e.addEventListener(g,f,!1):e.attachEvent&&e.attachEvent("on"+g,f))),p.add&&(p.add.call(e,d),d.handler.guid||(d.handler.guid=r.guid)),a?h.splice(h.delegateCount++,0,d):h.push(d),x.event.global[g]=!0);e=null}},remove:function(e,t,n,r,i){var o,a,s,l,u,c,p,f,d,h,g,m=x.hasData(e)&&x._data(e);if(m&&(c=m.events)){t=(t||"").match(T)||[""],u=t.length;while(u--)if(s=rt.exec(t[u])||[],d=g=s[1],h=(s[2]||"").split(".").sort(),d){p=x.event.special[d]||{},d=(r?p.delegateType:p.bindType)||d,f=c[d]||[],s=s[2]&&RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),l=o=f.length;while(o--)a=f[o],!i&&g!==a.origType||n&&n.guid!==a.guid||s&&!s.test(a.namespace
 )||r&&r!==a.selector&&("**"!==r||!a.selector)||(f.splice(o,1),a.selector&&f.delegateCount--,p.remove&&p.remove.call(e,a));l&&!f.length&&(p.teardown&&p.teardown.call(e,h,m.handle)!==!1||x.removeEvent(e,d,m.handle),delete c[d])}else for(d in c)x.event.remove(e,d+t[u],n,r,!0);x.isEmptyObject(c)&&(delete m.handle,x._removeData(e,"events"))}},trigger:function(n,r,i,o){var s,l,u,c,p,f,d,h=[i||a],g=v.call(n,"type")?n.type:n,m=v.call(n,"namespace")?n.namespace.split("."):[];if(u=f=i=i||a,3!==i.nodeType&&8!==i.nodeType&&!nt.test(g+x.event.triggered)&&(g.indexOf(".")>=0&&(m=g.split("."),g=m.shift(),m.sort()),l=0>g.indexOf(":")&&"on"+g,n=n[x.expando]?n:new x.Event(g,"object"==typeof n&&n),n.isTrigger=o?2:3,n.namespace=m.join("."),n.namespace_re=n.namespace?RegExp("(^|\\.)"+m.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,n.result=t,n.target||(n.target=i),r=null==r?[n]:x.makeArray(r,[n]),p=x.event.special[g]||{},o||!p.trigger||p.trigger.apply(i,r)!==!1)){if(!o&&!p.noBubble&&!x.isWindow(i)){for(c=p.deleg
 ateType||g,nt.test(c+g)||(u=u.parentNode);u;u=u.parentNode)h.push(u),f=u;f===(i.ownerDocument||a)&&h.push(f.defaultView||f.parentWindow||e)}d=0;while((u=h[d++])&&!n.isPropagationStopped())n.type=d>1?c:p.bindType||g,s=(x._data(u,"events")||{})[n.type]&&x._data(u,"handle"),s&&s.apply(u,r),s=l&&u[l],s&&x.acceptData(u)&&s.apply&&s.apply(u,r)===!1&&n.preventDefault();if(n.type=g,!o&&!n.isDefaultPrevented()&&(!p._default||p._default.apply(h.pop(),r)===!1)&&x.acceptData(i)&&l&&i[g]&&!x.isWindow(i)){f=i[l],f&&(i[l]=null),x.event.triggered=g;try{i[g]()}catch(y){}x.event.triggered=t,f&&(i[l]=f)}return n.result}},dispatch:function(e){e=x.event.fix(e);var n,r,i,o,a,s=[],l=g.call(arguments),u=(x._data(this,"events")||{})[e.type]||[],c=x.event.special[e.type]||{};if(l[0]=e,e.delegateTarget=this,!c.preDispatch||c.preDispatch.call(this,e)!==!1){s=x.event.handlers.call(this,e,u),n=0;while((o=s[n++])&&!e.isPropagationStopped()){e.currentTarget=o.elem,a=0;while((i=o.handlers[a++])&&!e.isImmediatePropa
 gationStopped())(!e.namespace_re||e.namespace_re.test(i.namespace))&&(e.handleObj=i,e.data=i.data,r=((x.event.special[i.origType]||{}).handle||i.handler).apply(o.elem,l),r!==t&&(e.result=r)===!1&&(e.preventDefault(),e.stopPropagation()))}return c.postDispatch&&c.postDispatch.call(this,e),e.result}},handlers:function(e,n){var r,i,o,a,s=[],l=n.delegateCount,u=e.target;if(l&&u.nodeType&&(!e.button||"click"!==e.type))for(;u!=this;u=u.parentNode||this)if(1===u.nodeType&&(u.disabled!==!0||"click"!==e.type)){for(o=[],a=0;l>a;a++)i=n[a],r=i.selector+" ",o[r]===t&&(o[r]=i.needsContext?x(r,this).index(u)>=0:x.find(r,this,null,[u]).length),o[r]&&o.push(i);o.length&&s.push({elem:u,handlers:o})}return n.length>l&&s.push({elem:this,handlers:n.slice(l)}),s},fix:function(e){if(e[x.expando])return e;var t,n,r,i=e.type,o=e,s=this.fixHooks[i];s||(this.fixHooks[i]=s=tt.test(i)?this.mouseHooks:et.test(i)?this.keyHooks:{}),r=s.props?this.props.concat(s.props):this.props,e=new x.Event(o),t=r.length;while(
 t--)n=r[t],e[n]=o[n];return e.target||(e.target=o.srcElement||a),3===e.target.nodeType&&(e.target=e.target.parentNode),e.metaKey=!!e.metaKey,s.filter?s.filter(e,o):e},props:"altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(e,t){return null==e.which&&(e.which=null!=t.charCode?t.charCode:t.keyCode),e}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(e,n){var r,i,o,s=n.button,l=n.fromElement;return null==e.pageX&&null!=n.clientX&&(i=e.target.ownerDocument||a,o=i.documentElement,r=i.body,e.pageX=n.clientX+(o&&o.scrollLeft||r&&r.scrollLeft||0)-(o&&o.clientLeft||r&&r.clientLeft||0),e.pageY=n.clientY+(o&&o.scrollTop||r&&r.scrollTop||0)-(o&&o.clientTop||r&&r.clientTop||0)),!e.relatedTarget&&l&&(e.relatedTarget=l===e.target?n.toElement:
 l),e.which||s===t||(e.which=1&s?1:2&s?3:4&s?2:0),e}},special:{load:{noBubble:!0},focus:{trigger:function(){if(this!==at()&&this.focus)try{return this.focus(),!1}catch(e){}},delegateType:"focusin"},blur:{trigger:function(){return this===at()&&this.blur?(this.blur(),!1):t},delegateType:"focusout"},click:{trigger:function(){return x.nodeName(this,"input")&&"checkbox"===this.type&&this.click?(this.click(),!1):t},_default:function(e){return x.nodeName(e.target,"a")}},beforeunload:{postDispatch:function(e){e.result!==t&&(e.originalEvent.returnValue=e.result)}}},simulate:function(e,t,n,r){var i=x.extend(new x.Event,n,{type:e,isSimulated:!0,originalEvent:{}});r?x.event.trigger(i,null,t):x.event.dispatch.call(t,i),i.isDefaultPrevented()&&n.preventDefault()}},x.removeEvent=a.removeEventListener?function(e,t,n){e.removeEventListener&&e.removeEventListener(t,n,!1)}:function(e,t,n){var r="on"+t;e.detachEvent&&(typeof e[r]===i&&(e[r]=null),e.detachEvent(r,n))},x.Event=function(e,n){return this in
 stanceof x.Event?(e&&e.type?(this.originalEvent=e,this.type=e.type,this.isDefaultPrevented=e.defaultPrevented||e.returnValue===!1||e.getPreventDefault&&e.getPreventDefault()?it:ot):this.type=e,n&&x.extend(this,n),this.timeStamp=e&&e.timeStamp||x.now(),this[x.expando]=!0,t):new x.Event(e,n)},x.Event.prototype={isDefaultPrevented:ot,isPropagationStopped:ot,isImmediatePropagationStopped:ot,preventDefault:function(){var e=this.originalEvent;this.isDefaultPrevented=it,e&&(e.preventDefault?e.preventDefault():e.returnValue=!1)},stopPropagation:function(){var e=this.originalEvent;this.isPropagationStopped=it,e&&(e.stopPropagation&&e.stopPropagation(),e.cancelBubble=!0)},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=it,this.stopPropagation()}},x.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(e,t){x.event.special[e]={delegateType:t,bindType:t,handle:function(e){var n,r=this,i=e.relatedTarget,o=e.handleObj;return(!i||i!==r&&!x.contains(r,i))&&(e.type=o.or
 igType,n=o.handler.apply(this,arguments),e.type=t),n}}}),x.support.submitBubbles||(x.event.special.submit={setup:function(){return x.nodeName(this,"form")?!1:(x.event.add(this,"click._submit keypress._submit",function(e){var n=e.target,r=x.nodeName(n,"input")||x.nodeName(n,"button")?n.form:t;r&&!x._data(r,"submitBubbles")&&(x.event.add(r,"submit._submit",function(e){e._submit_bubble=!0}),x._data(r,"submitBubbles",!0))}),t)},postDispatch:function(e){e._submit_bubble&&(delete e._submit_bubble,this.parentNode&&!e.isTrigger&&x.event.simulate("submit",this.parentNode,e,!0))},teardown:function(){return x.nodeName(this,"form")?!1:(x.event.remove(this,"._submit"),t)}}),x.support.changeBubbles||(x.event.special.change={setup:function(){return Z.test(this.nodeName)?(("checkbox"===this.type||"radio"===this.type)&&(x.event.add(this,"propertychange._change",function(e){"checked"===e.originalEvent.propertyName&&(this._just_changed=!0)}),x.event.add(this,"click._change",function(e){this._just_chan
 ged&&!e.isTrigger&&(this._just_changed=!1),x.event.simulate("change",this,e,!0)})),!1):(x.event.add(this,"beforeactivate._change",function(e){var t=e.target;Z.test(t.nodeName)&&!x._data(t,"changeBubbles")&&(x.event.add(t,"change._change",function(e){!this.parentNode||e.isSimulated||e.isTrigger||x.event.simulate("change",this.parentNode,e,!0)}),x._data(t,"changeBubbles",!0))}),t)},handle:function(e){var n=e.target;return this!==n||e.isSimulated||e.isTrigger||"radio"!==n.type&&"checkbox"!==n.type?e.handleObj.handler.apply(this,arguments):t},teardown:function(){return x.event.remove(this,"._change"),!Z.test(this.nodeName)}}),x.support.focusinBubbles||x.each({focus:"focusin",blur:"focusout"},function(e,t){var n=0,r=function(e){x.event.simulate(t,e.target,x.event.fix(e),!0)};x.event.special[t]={setup:function(){0===n++&&a.addEventListener(e,r,!0)},teardown:function(){0===--n&&a.removeEventListener(e,r,!0)}}}),x.fn.extend({on:function(e,n,r,i,o){var a,s;if("object"==typeof e){"string"!=ty
 peof n&&(r=r||n,n=t);for(a in e)this.on(a,n,r,e[a],o);return this}if(null==r&&null==i?(i=n,r=n=t):null==i&&("string"==typeof n?(i=r,r=t):(i=r,r=n,n=t)),i===!1)i=ot;else if(!i)return this;return 1===o&&(s=i,i=function(e){return x().off(e),s.apply(this,arguments)},i.guid=s.guid||(s.guid=x.guid++)),this.each(function(){x.event.add(this,e,i,r,n)})},one:function(e,t,n,r){return this.on(e,t,n,r,1)},off:function(e,n,r){var i,o;if(e&&e.preventDefault&&e.handleObj)return i=e.handleObj,x(e.delegateTarget).off(i.namespace?i.origType+"."+i.namespace:i.origType,i.selector,i.handler),this;if("object"==typeof e){for(o in e)this.off(o,n,e[o]);return this}return(n===!1||"function"==typeof n)&&(r=n,n=t),r===!1&&(r=ot),this.each(function(){x.event.remove(this,e,r,n)})},trigger:function(e,t){return this.each(function(){x.event.trigger(e,t,this)})},triggerHandler:function(e,n){var r=this[0];return r?x.event.trigger(e,n,r,!0):t}});var st=/^.[^:#\[\.,]*$/,lt=/^(?:parents|prev(?:Until|All))/,ut=x.expr.matc
 h.needsContext,ct={children:!0,contents:!0,next:!0,prev:!0};x.fn.extend({find:function(e){var t,n=[],r=this,i=r.length;if("string"!=typeof e)return this.pushStack(x(e).filter(function(){for(t=0;i>t;t++)if(x.contains(r[t],this))return!0}));for(t=0;i>t;t++)x.find(e,r[t],n);return n=this.pushStack(i>1?x.unique(n):n),n.selector=this.selector?this.selector+" "+e:e,n},has:function(e){var t,n=x(e,this),r=n.length;return this.filter(function(){for(t=0;r>t;t++)if(x.contains(this,n[t]))return!0})},not:function(e){return this.pushStack(ft(this,e||[],!0))},filter:function(e){return this.pushStack(ft(this,e||[],!1))},is:function(e){return!!ft(this,"string"==typeof e&&ut.test(e)?x(e):e||[],!1).length},closest:function(e,t){var n,r=0,i=this.length,o=[],a=ut.test(e)||"string"!=typeof e?x(e,t||this.context):0;for(;i>r;r++)for(n=this[r];n&&n!==t;n=n.parentNode)if(11>n.nodeType&&(a?a.index(n)>-1:1===n.nodeType&&x.find.matchesSelector(n,e))){n=o.push(n);break}return this.pushStack(o.length>1?x.unique(o
 ):o)},index:function(e){return e?"string"==typeof e?x.inArray(this[0],x(e)):x.inArray(e.jquery?e[0]:e,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(e,t){var n="string"==typeof e?x(e,t):x.makeArray(e&&e.nodeType?[e]:e),r=x.merge(this.get(),n);return this.pushStack(x.unique(r))},addBack:function(e){return this.add(null==e?this.prevObject:this.prevObject.filter(e))}});function pt(e,t){do e=e[t];while(e&&1!==e.nodeType);return e}x.each({parent:function(e){var t=e.parentNode;return t&&11!==t.nodeType?t:null},parents:function(e){return x.dir(e,"parentNode")},parentsUntil:function(e,t,n){return x.dir(e,"parentNode",n)},next:function(e){return pt(e,"nextSibling")},prev:function(e){return pt(e,"previousSibling")},nextAll:function(e){return x.dir(e,"nextSibling")},prevAll:function(e){return x.dir(e,"previousSibling")},nextUntil:function(e,t,n){return x.dir(e,"nextSibling",n)},prevUntil:function(e,t,n){return x.dir(e,"previousSibling",n)},siblings:function(e)
 {return x.sibling((e.parentNode||{}).firstChild,e)},children:function(e){return x.sibling(e.firstChild)},contents:function(e){return x.nodeName(e,"iframe")?e.contentDocument||e.contentWindow.document:x.merge([],e.childNodes)}},function(e,t){x.fn[e]=function(n,r){var i=x.map(this,t,n);return"Until"!==e.slice(-5)&&(r=n),r&&"string"==typeof r&&(i=x.filter(r,i)),this.length>1&&(ct[e]||(i=x.unique(i)),lt.test(e)&&(i=i.reverse())),this.pushStack(i)}}),x.extend({filter:function(e,t,n){var r=t[0];return n&&(e=":not("+e+")"),1===t.length&&1===r.nodeType?x.find.matchesSelector(r,e)?[r]:[]:x.find.matches(e,x.grep(t,function(e){return 1===e.nodeType}))},dir:function(e,n,r){var i=[],o=e[n];while(o&&9!==o.nodeType&&(r===t||1!==o.nodeType||!x(o).is(r)))1===o.nodeType&&i.push(o),o=o[n];return i},sibling:function(e,t){var n=[];for(;e;e=e.nextSibling)1===e.nodeType&&e!==t&&n.push(e);return n}});function ft(e,t,n){if(x.isFunction(t))return x.grep(e,function(e,r){return!!t.call(e,r,e)!==n});if(t.nodeTy
 pe)return x.grep(e,function(e){return e===t!==n});if("string"==typeof t){if(st.test(t))return x.filter(t,e,n);t=x.filter(t,e)}return x.grep(e,function(e){return x.inArray(e,t)>=0!==n})}function dt(e){var t=ht.split("|"),n=e.createDocumentFragment();if(n.createElement)while(t.length)n.createElement(t.pop());return n}var ht="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",gt=/ jQuery\d+="(?:null|\d+)"/g,mt=RegExp("<(?:"+ht+")[\\s/>]","i"),yt=/^\s+/,vt=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,bt=/<([\w:]+)/,xt=/<tbody/i,wt=/<|&#?\w+;/,Tt=/<(?:script|style|link)/i,Ct=/^(?:checkbox|radio)$/i,Nt=/checked\s*(?:[^=]|=\s*.checked.)/i,kt=/^$|\/(?:java|ecma)script/i,Et=/^true\/(.*)/,St=/^\s*<!(?:\[CDATA\[|--)|(?:\]\]|--)>\s*$/g,At={option:[1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],area:[1,"<map>","</map>"],param:[
 1,"<object>","</object>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],col:[2,"<table><tbody></tbody><colgroup>","</colgroup></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],_default:x.support.htmlSerialize?[0,"",""]:[1,"X<div>","</div>"]},jt=dt(a),Dt=jt.appendChild(a.createElement("div"));At.optgroup=At.option,At.tbody=At.tfoot=At.colgroup=At.caption=At.thead,At.th=At.td,x.fn.extend({text:function(e){return x.access(this,function(e){return e===t?x.text(this):this.empty().append((this[0]&&this[0].ownerDocument||a).createTextNode(e))},null,e,arguments.length)},append:function(){return this.domManip(arguments,function(e){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var t=Lt(this,e);t.appendChild(e)}})},prepend:function(){return this.domManip(arguments,function(e){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var t=Lt(this,e);t.insertBefore(e,t.firstChild)}})},before:function(){return this.domManip(arguments,func
 tion(e){this.parentNode&&this.parentNode.insertBefore(e,this)})},after:function(){return this.domManip(arguments,function(e){this.parentNode&&this.parentNode.insertBefore(e,this.nextSibling)})},remove:function(e,t){var n,r=e?x.filter(e,this):this,i=0;for(;null!=(n=r[i]);i++)t||1!==n.nodeType||x.cleanData(Ft(n)),n.parentNode&&(t&&x.contains(n.ownerDocument,n)&&_t(Ft(n,"script")),n.parentNode.removeChild(n));return this},empty:function(){var e,t=0;for(;null!=(e=this[t]);t++){1===e.nodeType&&x.cleanData(Ft(e,!1));while(e.firstChild)e.removeChild(e.firstChild);e.options&&x.nodeName(e,"select")&&(e.options.length=0)}return this},clone:function(e,t){return e=null==e?!1:e,t=null==t?e:t,this.map(function(){return x.clone(this,e,t)})},html:function(e){return x.access(this,function(e){var n=this[0]||{},r=0,i=this.length;if(e===t)return 1===n.nodeType?n.innerHTML.replace(gt,""):t;if(!("string"!=typeof e||Tt.test(e)||!x.support.htmlSerialize&&mt.test(e)||!x.support.leadingWhitespace&&yt.test(e)
 ||At[(bt.exec(e)||["",""])[1].toLowerCase()])){e=e.replace(vt,"<$1></$2>");try{for(;i>r;r++)n=this[r]||{},1===n.nodeType&&(x.cleanData(Ft(n,!1)),n.innerHTML=e);n=0}catch(o){}}n&&this.empty().append(e)},null,e,arguments.length)},replaceWith:function(){var e=x.map(this,function(e){return[e.nextSibling,e.parentNode]}),t=0;return this.domManip(arguments,function(n){var r=e[t++],i=e[t++];i&&(r&&r.parentNode!==i&&(r=this.nextSibling),x(this).remove(),i.insertBefore(n,r))},!0),t?this:this.remove()},detach:function(e){return this.remove(e,!0)},domManip:function(e,t,n){e=d.apply([],e);var r,i,o,a,s,l,u=0,c=this.length,p=this,f=c-1,h=e[0],g=x.isFunction(h);if(g||!(1>=c||"string"!=typeof h||x.support.checkClone)&&Nt.test(h))return this.each(function(r){var i=p.eq(r);g&&(e[0]=h.call(this,r,i.html())),i.domManip(e,t,n)});if(c&&(l=x.buildFragment(e,this[0].ownerDocument,!1,!n&&this),r=l.firstChild,1===l.childNodes.length&&(l=r),r)){for(a=x.map(Ft(l,"script"),Ht),o=a.length;c>u;u++)i=l,u!==f&&(i=x
 .clone(i,!0,!0),o&&x.merge(a,Ft(i,"script"))),t.call(this[u],i,u);if(o)for(s=a[a.length-1].ownerDocument,x.map(a,qt),u=0;o>u;u++)i=a[u],kt.test(i.type||"")&&!x._data(i,"globalEval")&&x.contains(s,i)&&(i.src?x._evalUrl(i.src):x.globalEval((i.text||i.textContent||i.innerHTML||"").replace(St,"")));l=r=null}return this}});function Lt(e,t){return x.nodeName(e,"table")&&x.nodeName(1===t.nodeType?t:t.firstChild,"tr")?e.getElementsByTagName("tbody")[0]||e.appendChild(e.ownerDocument.createElement("tbody")):e}function Ht(e){return e.type=(null!==x.find.attr(e,"type"))+"/"+e.type,e}function qt(e){var t=Et.exec(e.type);return t?e.type=t[1]:e.removeAttribute("type"),e}function _t(e,t){var n,r=0;for(;null!=(n=e[r]);r++)x._data(n,"globalEval",!t||x._data(t[r],"globalEval"))}function Mt(e,t){if(1===t.nodeType&&x.hasData(e)){var n,r,i,o=x._data(e),a=x._data(t,o),s=o.events;if(s){delete a.handle,a.events={};for(n in s)for(r=0,i=s[n].length;i>r;r++)x.event.add(t,n,s[n][r])}a.data&&(a.data=x.extend({}
 ,a.data))}}function Ot(e,t){var n,r,i;if(1===t.nodeType){if(n=t.nodeName.toLowerCase(),!x.support.noCloneEvent&&t[x.expando]){i=x._data(t);for(r in i.events)x.removeEvent(t,r,i.handle);t.removeAttribute(x.expando)}"script"===n&&t.text!==e.text?(Ht(t).text=e.text,qt(t)):"object"===n?(t.parentNode&&(t.outerHTML=e.outerHTML),x.support.html5Clone&&e.innerHTML&&!x.trim(t.innerHTML)&&(t.innerHTML=e.innerHTML)):"input"===n&&Ct.test(e.type)?(t.defaultChecked=t.checked=e.checked,t.value!==e.value&&(t.value=e.value)):"option"===n?t.defaultSelected=t.selected=e.defaultSelected:("input"===n||"textarea"===n)&&(t.defaultValue=e.defaultValue)}}x.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(e,t){x.fn[e]=function(e){var n,r=0,i=[],o=x(e),a=o.length-1;for(;a>=r;r++)n=r===a?this:this.clone(!0),x(o[r])[t](n),h.apply(i,n.get());return this.pushStack(i)}});function Ft(e,n){var r,o,a=0,s=typeof e.getElementsByTagName!==i?e.getElem
 entsByTagName(n||"*"):typeof e.querySelectorAll!==i?e.querySelectorAll(n||"*"):t;if(!s)for(s=[],r=e.childNodes||e;null!=(o=r[a]);a++)!n||x.nodeName(o,n)?s.push(o):x.merge(s,Ft(o,n));return n===t||n&&x.nodeName(e,n)?x.merge([e],s):s}function Bt(e){Ct.test(e.type)&&(e.defaultChecked=e.checked)}x.extend({clone:function(e,t,n){var r,i,o,a,s,l=x.contains(e.ownerDocument,e);if(x.support.html5Clone||x.isXMLDoc(e)||!mt.test("<"+e.nodeName+">")?o=e.cloneNode(!0):(Dt.innerHTML=e.outerHTML,Dt.removeChild(o=Dt.firstChild)),!(x.support.noCloneEvent&&x.support.noCloneChecked||1!==e.nodeType&&11!==e.nodeType||x.isXMLDoc(e)))for(r=Ft(o),s=Ft(e),a=0;null!=(i=s[a]);++a)r[a]&&Ot(i,r[a]);if(t)if(n)for(s=s||Ft(e),r=r||Ft(o),a=0;null!=(i=s[a]);a++)Mt(i,r[a]);else Mt(e,o);return r=Ft(o,"script"),r.length>0&&_t(r,!l&&Ft(e,"script")),r=s=i=null,o},buildFragment:function(e,t,n,r){var i,o,a,s,l,u,c,p=e.length,f=dt(t),d=[],h=0;for(;p>h;h++)if(o=e[h],o||0===o)if("object"===x.type(o))x.merge(d,o.nodeType?[o]:o);
 else if(wt.test(o)){s=s||f.appendChild(t.createElement("div")),l=(bt.exec(o)||["",""])[1].toLowerCase(),c=At[l]||At._default,s.innerHTML=c[1]+o.replace(vt,"<$1></$2>")+c[2],i=c[0];while(i--)s=s.lastChild;if(!x.support.leadingWhitespace&&yt.test(o)&&d.push(t.createTextNode(yt.exec(o)[0])),!x.support.tbody){o="table"!==l||xt.test(o)?"<table>"!==c[1]||xt.test(o)?0:s:s.firstChild,i=o&&o.childNodes.length;while(i--)x.nodeName(u=o.childNodes[i],"tbody")&&!u.childNodes.length&&o.removeChild(u)}x.merge(d,s.childNodes),s.textContent="";while(s.firstChild)s.removeChild(s.firstChild);s=f.lastChild}else d.push(t.createTextNode(o));s&&f.removeChild(s),x.support.appendChecked||x.grep(Ft(d,"input"),Bt),h=0;while(o=d[h++])if((!r||-1===x.inArray(o,r))&&(a=x.contains(o.ownerDocument,o),s=Ft(f.appendChild(o),"script"),a&&_t(s),n)){i=0;while(o=s[i++])kt.test(o.type||"")&&n.push(o)}return s=null,f},cleanData:function(e,t){var n,r,o,a,s=0,l=x.expando,u=x.cache,c=x.support.deleteExpando,f=x.event.special;
 for(;null!=(n=e[s]);s++)if((t||x.acceptData(n))&&(o=n[l],a=o&&u[o])){if(a.events)for(r in a.events)f[r]?x.event.remove(n,r):x.removeEvent(n,r,a.handle);
-u[o]&&(delete u[o],c?delete n[l]:typeof n.removeAttribute!==i?n.removeAttribute(l):n[l]=null,p.push(o))}},_evalUrl:function(e){return x.ajax({url:e,type:"GET",dataType:"script",async:!1,global:!1,"throws":!0})}}),x.fn.extend({wrapAll:function(e){if(x.isFunction(e))return this.each(function(t){x(this).wrapAll(e.call(this,t))});if(this[0]){var t=x(e,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&t.insertBefore(this[0]),t.map(function(){var e=this;while(e.firstChild&&1===e.firstChild.nodeType)e=e.firstChild;return e}).append(this)}return this},wrapInner:function(e){return x.isFunction(e)?this.each(function(t){x(this).wrapInner(e.call(this,t))}):this.each(function(){var t=x(this),n=t.contents();n.length?n.wrapAll(e):t.append(e)})},wrap:function(e){var t=x.isFunction(e);return this.each(function(n){x(this).wrapAll(t?e.call(this,n):e)})},unwrap:function(){return this.parent().each(function(){x.nodeName(this,"body")||x(this).replaceWith(this.childNodes)}).end()}});var Pt,Rt,Wt,
 $t=/alpha\([^)]*\)/i,It=/opacity\s*=\s*([^)]*)/,zt=/^(top|right|bottom|left)$/,Xt=/^(none|table(?!-c[ea]).+)/,Ut=/^margin/,Vt=RegExp("^("+w+")(.*)$","i"),Yt=RegExp("^("+w+")(?!px)[a-z%]+$","i"),Jt=RegExp("^([+-])=("+w+")","i"),Gt={BODY:"block"},Qt={position:"absolute",visibility:"hidden",display:"block"},Kt={letterSpacing:0,fontWeight:400},Zt=["Top","Right","Bottom","Left"],en=["Webkit","O","Moz","ms"];function tn(e,t){if(t in e)return t;var n=t.charAt(0).toUpperCase()+t.slice(1),r=t,i=en.length;while(i--)if(t=en[i]+n,t in e)return t;return r}function nn(e,t){return e=t||e,"none"===x.css(e,"display")||!x.contains(e.ownerDocument,e)}function rn(e,t){var n,r,i,o=[],a=0,s=e.length;for(;s>a;a++)r=e[a],r.style&&(o[a]=x._data(r,"olddisplay"),n=r.style.display,t?(o[a]||"none"!==n||(r.style.display=""),""===r.style.display&&nn(r)&&(o[a]=x._data(r,"olddisplay",ln(r.nodeName)))):o[a]||(i=nn(r),(n&&"none"!==n||!i)&&x._data(r,"olddisplay",i?n:x.css(r,"display"))));for(a=0;s>a;a++)r=e[a],r.style
 &&(t&&"none"!==r.style.display&&""!==r.style.display||(r.style.display=t?o[a]||"":"none"));return e}x.fn.extend({css:function(e,n){return x.access(this,function(e,n,r){var i,o,a={},s=0;if(x.isArray(n)){for(o=Rt(e),i=n.length;i>s;s++)a[n[s]]=x.css(e,n[s],!1,o);return a}return r!==t?x.style(e,n,r):x.css(e,n)},e,n,arguments.length>1)},show:function(){return rn(this,!0)},hide:function(){return rn(this)},toggle:function(e){var t="boolean"==typeof e;return this.each(function(){(t?e:nn(this))?x(this).show():x(this).hide()})}}),x.extend({cssHooks:{opacity:{get:function(e,t){if(t){var n=Wt(e,"opacity");return""===n?"1":n}}}},cssNumber:{columnCount:!0,fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":x.support.cssFloat?"cssFloat":"styleFloat"},style:function(e,n,r,i){if(e&&3!==e.nodeType&&8!==e.nodeType&&e.style){var o,a,s,l=x.camelCase(n),u=e.style;if(n=x.cssProps[l]||(x.cssProps[l]=tn(u,l)),s=x.cssHooks[n]||x.cssHooks[l],r===t)re
 turn s&&"get"in s&&(o=s.get(e,!1,i))!==t?o:u[n];if(a=typeof r,"string"===a&&(o=Jt.exec(r))&&(r=(o[1]+1)*o[2]+parseFloat(x.css(e,n)),a="number"),!(null==r||"number"===a&&isNaN(r)||("number"!==a||x.cssNumber[l]||(r+="px"),x.support.clearCloneStyle||""!==r||0!==n.indexOf("background")||(u[n]="inherit"),s&&"set"in s&&(r=s.set(e,r,i))===t)))try{u[n]=r}catch(c){}}},css:function(e,n,r,i){var o,a,s,l=x.camelCase(n);return n=x.cssProps[l]||(x.cssProps[l]=tn(e.style,l)),s=x.cssHooks[n]||x.cssHooks[l],s&&"get"in s&&(a=s.get(e,!0,r)),a===t&&(a=Wt(e,n,i)),"normal"===a&&n in Kt&&(a=Kt[n]),""===r||r?(o=parseFloat(a),r===!0||x.isNumeric(o)?o||0:a):a}}),e.getComputedStyle?(Rt=function(t){return e.getComputedStyle(t,null)},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),l=s?s.getPropertyValue(n)||s[n]:t,u=e.style;return s&&(""!==l||x.contains(e.ownerDocument,e)||(l=x.style(e,n)),Yt.test(l)&&Ut.test(n)&&(i=u.width,o=u.minWidth,a=u.maxWidth,u.minWidth=u.maxWidth=u.width=l,l=s.width,u.width=i,u.minWidth=o,u.max
 Width=a)),l}):a.documentElement.currentStyle&&(Rt=function(e){return e.currentStyle},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),l=s?s[n]:t,u=e.style;return null==l&&u&&u[n]&&(l=u[n]),Yt.test(l)&&!zt.test(n)&&(i=u.left,o=e.runtimeStyle,a=o&&o.left,a&&(o.left=e.currentStyle.left),u.left="fontSize"===n?"1em":l,l=u.pixelLeft+"px",u.left=i,a&&(o.left=a)),""===l?"auto":l});function on(e,t,n){var r=Vt.exec(t);return r?Math.max(0,r[1]-(n||0))+(r[2]||"px"):t}function an(e,t,n,r,i){var o=n===(r?"border":"content")?4:"width"===t?1:0,a=0;for(;4>o;o+=2)"margin"===n&&(a+=x.css(e,n+Zt[o],!0,i)),r?("content"===n&&(a-=x.css(e,"padding"+Zt[o],!0,i)),"margin"!==n&&(a-=x.css(e,"border"+Zt[o]+"Width",!0,i))):(a+=x.css(e,"padding"+Zt[o],!0,i),"padding"!==n&&(a+=x.css(e,"border"+Zt[o]+"Width",!0,i)));return a}function sn(e,t,n){var r=!0,i="width"===t?e.offsetWidth:e.offsetHeight,o=Rt(e),a=x.support.boxSizing&&"border-box"===x.css(e,"boxSizing",!1,o);if(0>=i||null==i){if(i=Wt(e,t,o),(0>i||null==i)&&(i=e.style
 [t]),Yt.test(i))return i;r=a&&(x.support.boxSizingReliable||i===e.style[t]),i=parseFloat(i)||0}return i+an(e,t,n||(a?"border":"content"),r,o)+"px"}function ln(e){var t=a,n=Gt[e];return n||(n=un(e,t),"none"!==n&&n||(Pt=(Pt||x("<iframe frameborder='0' width='0' height='0'/>").css("cssText","display:block !important")).appendTo(t.documentElement),t=(Pt[0].contentWindow||Pt[0].contentDocument).document,t.write("<!doctype html><html><body>"),t.close(),n=un(e,t),Pt.detach()),Gt[e]=n),n}function un(e,t){var n=x(t.createElement(e)).appendTo(t.body),r=x.css(n[0],"display");return n.remove(),r}x.each(["height","width"],function(e,n){x.cssHooks[n]={get:function(e,r,i){return r?0===e.offsetWidth&&Xt.test(x.css(e,"display"))?x.swap(e,Qt,function(){return sn(e,n,i)}):sn(e,n,i):t},set:function(e,t,r){var i=r&&Rt(e);return on(e,t,r?an(e,n,r,x.support.boxSizing&&"border-box"===x.css(e,"boxSizing",!1,i),i):0)}}}),x.support.opacity||(x.cssHooks.opacity={get:function(e,t){return It.test((t&&e.currentSt
 yle?e.currentStyle.filter:e.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":t?"1":""},set:function(e,t){var n=e.style,r=e.currentStyle,i=x.isNumeric(t)?"alpha(opacity="+100*t+")":"",o=r&&r.filter||n.filter||"";n.zoom=1,(t>=1||""===t)&&""===x.trim(o.replace($t,""))&&n.removeAttribute&&(n.removeAttribute("filter"),""===t||r&&!r.filter)||(n.filter=$t.test(o)?o.replace($t,i):o+" "+i)}}),x(function(){x.support.reliableMarginRight||(x.cssHooks.marginRight={get:function(e,n){return n?x.swap(e,{display:"inline-block"},Wt,[e,"marginRight"]):t}}),!x.support.pixelPosition&&x.fn.position&&x.each(["top","left"],function(e,n){x.cssHooks[n]={get:function(e,r){return r?(r=Wt(e,n),Yt.test(r)?x(e).position()[n]+"px":r):t}}})}),x.expr&&x.expr.filters&&(x.expr.filters.hidden=function(e){return 0>=e.offsetWidth&&0>=e.offsetHeight||!x.support.reliableHiddenOffsets&&"none"===(e.style&&e.style.display||x.css(e,"display"))},x.expr.filters.visible=function(e){return!x.expr.filters.hidden(e)}),x.each({margin:
 "",padding:"",border:"Width"},function(e,t){x.cssHooks[e+t]={expand:function(n){var r=0,i={},o="string"==typeof n?n.split(" "):[n];for(;4>r;r++)i[e+Zt[r]+t]=o[r]||o[r-2]||o[0];return i}},Ut.test(e)||(x.cssHooks[e+t].set=on)});var cn=/%20/g,pn=/\[\]$/,fn=/\r?\n/g,dn=/^(?:submit|button|image|reset|file)$/i,hn=/^(?:input|select|textarea|keygen)/i;x.fn.extend({serialize:function(){return x.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var e=x.prop(this,"elements");return e?x.makeArray(e):this}).filter(function(){var e=this.type;return this.name&&!x(this).is(":disabled")&&hn.test(this.nodeName)&&!dn.test(e)&&(this.checked||!Ct.test(e))}).map(function(e,t){var n=x(this).val();return null==n?null:x.isArray(n)?x.map(n,function(e){return{name:t.name,value:e.replace(fn,"\r\n")}}):{name:t.name,value:n.replace(fn,"\r\n")}}).get()}}),x.param=function(e,n){var r,i=[],o=function(e,t){t=x.isFunction(t)?t():null==t?"":t,i[i.length]=encodeURIComponent(e)+"="+encod
 eURIComponent(t)};if(n===t&&(n=x.ajaxSettings&&x.ajaxSettings.traditional),x.isArray(e)||e.jquery&&!x.isPlainObject(e))x.each(e,function(){o(this.name,this.value)});else for(r in e)gn(r,e[r],n,o);return i.join("&").replace(cn,"+")};function gn(e,t,n,r){var i;if(x.isArray(t))x.each(t,function(t,i){n||pn.test(e)?r(e,i):gn(e+"["+("object"==typeof i?t:"")+"]",i,n,r)});else if(n||"object"!==x.type(t))r(e,t);else for(i in t)gn(e+"["+i+"]",t[i],n,r)}x.each("blur focus focusin focusout load resize scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(e,t){x.fn[t]=function(e,n){return arguments.length>0?this.on(t,null,e,n):this.trigger(t)}}),x.fn.extend({hover:function(e,t){return this.mouseenter(e).mouseleave(t||e)},bind:function(e,t,n){return this.on(e,null,t,n)},unbind:function(e,t){return this.off(e,null,t)},delegate:function(e,t,n,r){return this.on(t,e,n,r)},und
 elegate:function(e,t,n){return 1===arguments.length?this.off(e,"**"):this.off(t,e||"**",n)}});var mn,yn,vn=x.now(),bn=/\?/,xn=/#.*$/,wn=/([?&])_=[^&]*/,Tn=/^(.*?):[ \t]*([^\r\n]*)\r?$/gm,Cn=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Nn=/^(?:GET|HEAD)$/,kn=/^\/\//,En=/^([\w.+-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,Sn=x.fn.load,An={},jn={},Dn="*/".concat("*");try{yn=o.href}catch(Ln){yn=a.createElement("a"),yn.href="",yn=yn.href}mn=En.exec(yn.toLowerCase())||[];function Hn(e){return function(t,n){"string"!=typeof t&&(n=t,t="*");var r,i=0,o=t.toLowerCase().match(T)||[];if(x.isFunction(n))while(r=o[i++])"+"===r[0]?(r=r.slice(1)||"*",(e[r]=e[r]||[]).unshift(n)):(e[r]=e[r]||[]).push(n)}}function qn(e,n,r,i){var o={},a=e===jn;function s(l){var u;return o[l]=!0,x.each(e[l]||[],function(e,l){var c=l(n,r,i);return"string"!=typeof c||a||o[c]?a?!(u=c):t:(n.dataTypes.unshift(c),s(c),!1)}),u}return s(n.dataTypes[0])||!o["*"]&&s("*")}function _n(e,n){var r,i,o=x.ajaxSettings.flatOpti
 ons||{};for(i in n)n[i]!==t&&((o[i]?e:r||(r={}))[i]=n[i]);return r&&x.extend(!0,e,r),e}x.fn.load=function(e,n,r){if("string"!=typeof e&&Sn)return Sn.apply(this,arguments);var i,o,a,s=this,l=e.indexOf(" ");return l>=0&&(i=e.slice(l,e.length),e=e.slice(0,l)),x.isFunction(n)?(r=n,n=t):n&&"object"==typeof n&&(a="POST"),s.length>0&&x.ajax({url:e,type:a,dataType:"html",data:n}).done(function(e){o=arguments,s.html(i?x("<div>").append(x.parseHTML(e)).find(i):e)}).complete(r&&function(e,t){s.each(r,o||[e.responseText,t,e])}),this},x.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){x.fn[t]=function(e){return this.on(t,e)}}),x.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:yn,type:"GET",isLocal:Cn.test(mn[1]),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Dn,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents
 :{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText",json:"responseJSON"},converters:{"* text":String,"text html":!0,"text json":x.parseJSON,"text xml":x.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(e,t){return t?_n(_n(e,x.ajaxSettings),t):_n(x.ajaxSettings,e)},ajaxPrefilter:Hn(An),ajaxTransport:Hn(jn),ajax:function(e,n){"object"==typeof e&&(n=e,e=t),n=n||{};var r,i,o,a,s,l,u,c,p=x.ajaxSetup({},n),f=p.context||p,d=p.context&&(f.nodeType||f.jquery)?x(f):x.event,h=x.Deferred(),g=x.Callbacks("once memory"),m=p.statusCode||{},y={},v={},b=0,w="canceled",C={readyState:0,getResponseHeader:function(e){var t;if(2===b){if(!c){c={};while(t=Tn.exec(a))c[t[1].toLowerCase()]=t[2]}t=c[e.toLowerCase()]}return null==t?null:t},getAllResponseHeaders:function(){return 2===b?a:null},setRequestHeader:function(e,t){var n=e.toLowerCase();return b||(e=v[n]=v[n]||e,y[e]=t),this},overrideMimeType:function(e){return b||(p.mimeType=e),this},statusCode:funct
 ion(e){var t;if(e)if(2>b)for(t in e)m[t]=[m[t],e[t]];else C.always(e[C.status]);return this},abort:function(e){var t=e||w;return u&&u.abort(t),k(0,t),this}};if(h.promise(C).complete=g.add,C.success=C.done,C.error=C.fail,p.url=((e||p.url||yn)+"").replace(xn,"").replace(kn,mn[1]+"//"),p.type=n.method||n.type||p.method||p.type,p.dataTypes=x.trim(p.dataType||"*").toLowerCase().match(T)||[""],null==p.crossDomain&&(r=En.exec(p.url.toLowerCase()),p.crossDomain=!(!r||r[1]===mn[1]&&r[2]===mn[2]&&(r[3]||("http:"===r[1]?"80":"443"))===(mn[3]||("http:"===mn[1]?"80":"443")))),p.data&&p.processData&&"string"!=typeof p.data&&(p.data=x.param(p.data,p.traditional)),qn(An,p,n,C),2===b)return C;l=p.global,l&&0===x.active++&&x.event.trigger("ajaxStart"),p.type=p.type.toUpperCase(),p.hasContent=!Nn.test(p.type),o=p.url,p.hasContent||(p.data&&(o=p.url+=(bn.test(o)?"&":"?")+p.data,delete p.data),p.cache===!1&&(p.url=wn.test(o)?o.replace(wn,"$1_="+vn++):o+(bn.test(o)?"&":"?")+"_="+vn++)),p.ifModified&&(x.l
 astModified[o]&&C.setRequestHeader("If-Modified-Since",x.lastModified[o]),x.etag[o]&&C.setRequestHeader("If-None-Match",x.etag[o])),(p.data&&p.hasContent&&p.contentType!==!1||n.contentType)&&C.setRequestHeader("Content-Type",p.contentType),C.setRequestHeader("Accept",p.dataTypes[0]&&p.accepts[p.dataTypes[0]]?p.accepts[p.dataTypes[0]]+("*"!==p.dataTypes[0]?", "+Dn+"; q=0.01":""):p.accepts["*"]);for(i in p.headers)C.setRequestHeader(i,p.headers[i]);if(p.beforeSend&&(p.beforeSend.call(f,C,p)===!1||2===b))return C.abort();w="abort";for(i in{success:1,error:1,complete:1})C[i](p[i]);if(u=qn(jn,p,n,C)){C.readyState=1,l&&d.trigger("ajaxSend",[C,p]),p.async&&p.timeout>0&&(s=setTimeout(function(){C.abort("timeout")},p.timeout));try{b=1,u.send(y,k)}catch(N){if(!(2>b))throw N;k(-1,N)}}else k(-1,"No Transport");function k(e,n,r,i){var c,y,v,w,T,N=n;2!==b&&(b=2,s&&clearTimeout(s),u=t,a=i||"",C.readyState=e>0?4:0,c=e>=200&&300>e||304===e,r&&(w=Mn(p,C,r)),w=On(p,w,C,c),c?(p.ifModified&&(T=C.getResp
 onseHeader("Last-Modified"),T&&(x.lastModified[o]=T),T=C.getResponseHeader("etag"),T&&(x.etag[o]=T)),204===e||"HEAD"===p.type?N="nocontent":304===e?N="notmodified":(N=w.state,y=w.data,v=w.error,c=!v)):(v=N,(e||!N)&&(N="error",0>e&&(e=0))),C.status=e,C.statusText=(n||N)+"",c?h.resolveWith(f,[y,N,C]):h.rejectWith(f,[C,N,v]),C.statusCode(m),m=t,l&&d.trigger(c?"ajaxSuccess":"ajaxError",[C,p,c?y:v]),g.fireWith(f,[C,N]),l&&(d.trigger("ajaxComplete",[C,p]),--x.active||x.event.trigger("ajaxStop")))}return C},getJSON:function(e,t,n){return x.get(e,t,n,"json")},getScript:function(e,n){return x.get(e,t,n,"script")}}),x.each(["get","post"],function(e,n){x[n]=function(e,r,i,o){return x.isFunction(r)&&(o=o||i,i=r,r=t),x.ajax({url:e,type:n,dataType:o,data:r,success:i})}});function Mn(e,n,r){var i,o,a,s,l=e.contents,u=e.dataTypes;while("*"===u[0])u.shift(),o===t&&(o=e.mimeType||n.getResponseHeader("Content-Type"));if(o)for(s in l)if(l[s]&&l[s].test(o)){u.unshift(s);break}if(u[0]in r)a=u[0];else{for
 (s in r){if(!u[0]||e.converters[s+" "+u[0]]){a=s;break}i||(i=s)}a=a||i}return a?(a!==u[0]&&u.unshift(a),r[a]):t}function On(e,t,n,r){var i,o,a,s,l,u={},c=e.dataTypes.slice();if(c[1])for(a in e.converters)u[a.toLowerCase()]=e.converters[a];o=c.shift();while(o)if(e.responseFields[o]&&(n[e.responseFields[o]]=t),!l&&r&&e.dataFilter&&(t=e.dataFilter(t,e.dataType)),l=o,o=c.shift())if("*"===o)o=l;else if("*"!==l&&l!==o){if(a=u[l+" "+o]||u["* "+o],!a)for(i in u)if(s=i.split(" "),s[1]===o&&(a=u[l+" "+s[0]]||u["* "+s[0]])){a===!0?a=u[i]:u[i]!==!0&&(o=s[0],c.unshift(s[1]));break}if(a!==!0)if(a&&e["throws"])t=a(t);else try{t=a(t)}catch(p){return{state:"parsererror",error:a?p:"No conversion from "+l+" to "+o}}}return{state:"success",data:t}}x.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/(?:java|ecma)script/},converters:{"text script":function(e){return x.globalEval(e),e}}}),x.ajaxPrefilter("script",funct
 ion(e){e.cache===t&&(e.cache=!1),e.crossDomain&&(e.type="GET",e.global=!1)}),x.ajaxTransport("script",function(e){if(e.crossDomain){var n,r=a.head||x("head")[0]||a.documentElement;return{send:function(t,i){n=a.createElement("script"),n.async=!0,e.scriptCharset&&(n.charset=e.scriptCharset),n.src=e.url,n.onload=n.onreadystatechange=function(e,t){(t||!n.readyState||/loaded|complete/.test(n.readyState))&&(n.onload=n.onreadystatechange=null,n.parentNode&&n.parentNode.removeChild(n),n=null,t||i(200,"success"))},r.insertBefore(n,r.firstChild)},abort:function(){n&&n.onload(t,!0)}}}});var Fn=[],Bn=/(=)\?(?=&|$)|\?\?/;x.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=Fn.pop()||x.expando+"_"+vn++;return this[e]=!0,e}}),x.ajaxPrefilter("json jsonp",function(n,r,i){var o,a,s,l=n.jsonp!==!1&&(Bn.test(n.url)?"url":"string"==typeof n.data&&!(n.contentType||"").indexOf("application/x-www-form-urlencoded")&&Bn.test(n.data)&&"data");return l||"jsonp"===n.dataTypes[0]?(o=n.jsonpCallback=x.is
 Function(n.jsonpCallback)?n.jsonpCallback():n.jsonpCallback,l?n[l]=n[l].replace(Bn,"$1"+o):n.jsonp!==!1&&(n.url+=(bn.test(n.url)?"&":"?")+n.jsonp+"="+o),n.converters["script json"]=function(){return s||x.error(o+" was not called"),s[0]},n.dataTypes[0]="json",a=e[o],e[o]=function(){s=arguments},i.always(function(){e[o]=a,n[o]&&(n.jsonpCallback=r.jsonpCallback,Fn.push(o)),s&&x.isFunction(a)&&a(s[0]),s=a=t}),"script"):t});var Pn,Rn,Wn=0,$n=e.ActiveXObject&&function(){var e;for(e in Pn)Pn[e](t,!0)};function In(){try{return new e.XMLHttpRequest}catch(t){}}function zn(){try{return new e.ActiveXObject("Microsoft.XMLHTTP")}catch(t){}}x.ajaxSettings.xhr=e.ActiveXObject?function(){return!this.isLocal&&In()||zn()}:In,Rn=x.ajaxSettings.xhr(),x.support.cors=!!Rn&&"withCredentials"in Rn,Rn=x.support.ajax=!!Rn,Rn&&x.ajaxTransport(function(n){if(!n.crossDomain||x.support.cors){var r;return{send:function(i,o){var a,s,l=n.xhr();if(n.username?l.open(n.type,n.url,n.async,n.username,n.password):l.open(n
 .type,n.url,n.async),n.xhrFields)for(s in n.xhrFields)l[s]=n.xhrFields[s];n.mimeType&&l.overrideMimeType&&l.overrideMimeType(n.mimeType),n.crossDomain||i["X-Requested-With"]||(i["X-Requested-With"]="XMLHttpRequest");try{for(s in i)l.setRequestHeader(s,i[s])}catch(u){}l.send(n.hasContent&&n.data||null),r=function(e,i){var s,u,c,p;try{if(r&&(i||4===l.readyState))if(r=t,a&&(l.onreadystatechange=x.noop,$n&&delete Pn[a]),i)4!==l.readyState&&l.abort();else{p={},s=l.status,u=l.getAllResponseHeaders(),"string"==typeof l.responseText&&(p.text=l.responseText);try{c=l.statusText}catch(f){c=""}s||!n.isLocal||n.crossDomain?1223===s&&(s=204):s=p.text?200:404}}catch(d){i||o(-1,d)}p&&o(s,c,p,u)},n.async?4===l.readyState?setTimeout(r):(a=++Wn,$n&&(Pn||(Pn={},x(e).unload($n)),Pn[a]=r),l.onreadystatechange=r):r()},abort:function(){r&&r(t,!0)}}}});var Xn,Un,Vn=/^(?:toggle|show|hide)$/,Yn=RegExp("^(?:([+-])=|)("+w+")([a-z%]*)$","i"),Jn=/queueHooks$/,Gn=[nr],Qn={"*":[function(e,t){var n=this.createTween(
 e,t),r=n.cur(),i=Yn.exec(t),o=i&&i[3]||(x.cssNumber[e]?"":"px"),a=(x.cssNumber[e]||"px"!==o&&+r)&&Yn.exec(x.css(n.elem,e)),s=1,l=20;if(a&&a[3]!==o){o=o||a[3],i=i||[],a=+r||1;do s=s||".5",a/=s,x.style(n.elem,e,a+o);while(s!==(s=n.cur()/r)&&1!==s&&--l)}return i&&(a=n.start=+a||+r||0,n.unit=o,n.end=i[1]?a+(i[1]+1)*i[2]:+i[2]),n}]};function Kn(){return setTimeout(function(){Xn=t}),Xn=x.now()}function Zn(e,t,n){var r,i=(Qn[t]||[]).concat(Qn["*"]),o=0,a=i.length;for(;a>o;o++)if(r=i[o].call(n,t,e))return r}function er(e,t,n){var r,i,o=0,a=Gn.length,s=x.Deferred().always(function(){delete l.elem}),l=function(){if(i)return!1;var t=Xn||Kn(),n=Math.max(0,u.startTime+u.duration-t),r=n/u.duration||0,o=1-r,a=0,l=u.tweens.length;for(;l>a;a++)u.tweens[a].run(o);return s.notifyWith(e,[u,o,n]),1>o&&l?n:(s.resolveWith(e,[u]),!1)},u=s.promise({elem:e,props:x.extend({},t),opts:x.extend(!0,{specialEasing:{}},n),originalProperties:t,originalOptions:n,startTime:Xn||Kn(),duration:n.duration,tweens:[],create
 Tween:function(t,n){var r=x.Tween(e,u.opts,t,n,u.opts.specialEasing[t]||u.opts.easing);return u.tweens.push(r),r},stop:function(t){var n=0,r=t?u.tweens.length:0;if(i)return this;for(i=!0;r>n;n++)u.tweens[n].run(1);return t?s.resolveWith(e,[u,t]):s.rejectWith(e,[u,t]),this}}),c=u.props;for(tr(c,u.opts.specialEasing);a>o;o++)if(r=Gn[o].call(u,e,c,u.opts))return r;return x.map(c,Zn,u),x.isFunction(u.opts.start)&&u.opts.start.call(e,u),x.fx.timer(x.extend(l,{elem:e,anim:u,queue:u.opts.queue})),u.progress(u.opts.progress).done(u.opts.done,u.opts.complete).fail(u.opts.fail).always(u.opts.always)}function tr(e,t){var n,r,i,o,a;for(n in e)if(r=x.camelCase(n),i=t[r],o=e[n],x.isArray(o)&&(i=o[1],o=e[n]=o[0]),n!==r&&(e[r]=o,delete e[n]),a=x.cssHooks[r],a&&"expand"in a){o=a.expand(o),delete e[r];for(n in o)n in e||(e[n]=o[n],t[n]=i)}else t[r]=i}x.Animation=x.extend(er,{tweener:function(e,t){x.isFunction(e)?(t=e,e=["*"]):e=e.split(" ");var n,r=0,i=e.length;for(;i>r;r++)n=e[r],Qn[n]=Qn[n]||[],Qn[
 n].unshift(t)},prefilter:function(e,t){t?Gn.unshift(e):Gn.push(e)}});function nr(e,t,n){var r,i,o,a,s,l,u=this,c={},p=e.style,f=e.nodeType&&nn(e),d=x._data(e,"fxshow");n.queue||(s=x._queueHooks(e,"fx"),null==s.unqueued&&(s.unqueued=0,l=s.empty.fire,s.empty.fire=function(){s.unqueued||l()}),s.unqueued++,u.always(function(){u.always(function(){s.unqueued--,x.queue(e,"fx").length||s.empty.fire()})})),1===e.nodeType&&("height"in t||"width"in t)&&(n.overflow=[p.overflow,p.overflowX,p.overflowY],"inline"===x.css(e,"display")&&"none"===x.css(e,"float")&&(x.support.inlineBlockNeedsLayout&&"inline"!==ln(e.nodeName)?p.zoom=1:p.display="inline-block")),n.overflow&&(p.overflow="hidden",x.support.shrinkWrapBlocks||u.always(function(){p.overflow=n.overflow[0],p.overflowX=n.overflow[1],p.overflowY=n.overflow[2]}));for(r in t)if(i=t[r],Vn.exec(i)){if(delete t[r],o=o||"toggle"===i,i===(f?"hide":"show"))continue;c[r]=d&&d[r]||x.style(e,r)}if(!x.isEmptyObject(c)){d?"hidden"in d&&(f=d.hidden):d=x._data
 (e,"fxshow",{}),o&&(d.hidden=!f),f?x(e).show():u.done(function(){x(e).hide()}),u.done(function(){var t;x._removeData(e,"fxshow");for(t in c)x.style(e,t,c[t])});for(r in c)a=Zn(f?d[r]:0,r,u),r in d||(d[r]=a.start,f&&(a.end=a.start,a.start="width"===r||"height"===r?1:0))}}function rr(e,t,n,r,i){return new rr.prototype.init(e,t,n,r,i)}x.Tween=rr,rr.prototype={constructor:rr,init:function(e,t,n,r,i,o){this.elem=e,this.prop=n,this.easing=i||"swing",this.options=t,this.start=this.now=this.cur(),this.end=r,this.unit=o||(x.cssNumber[n]?"":"px")},cur:function(){var e=rr.propHooks[this.prop];return e&&e.get?e.get(this):rr.propHooks._default.get(this)},run:function(e){var t,n=rr.propHooks[this.prop];return this.pos=t=this.options.duration?x.easing[this.easing](e,this.options.duration*e,0,1,this.options.duration):e,this.now=(this.end-this.start)*t+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),n&&n.set?n.set(this):rr.propHooks._default.set(this),this}},rr.prototyp
 e.init.prototype=rr.prototype,rr.propHooks={_default:{get:function(e){var t;return null==e.elem[e.prop]||e.elem.style&&null!=e.elem.style[e.prop]?(t=x.css(e.elem,e.prop,""),t&&"auto"!==t?t:0):e.elem[e.prop]},set:function(e){x.fx.step[e.prop]?x.fx.step[e.prop](e):e.elem.style&&(null!=e.elem.style[x.cssProps[e.prop]]||x.cssHooks[e.prop])?x.style(e.elem,e.prop,e.now+e.unit):e.elem[e.prop]=e.now}}},rr.propHooks.scrollTop=rr.propHooks.scrollLeft={set:function(e){e.elem.nodeType&&e.elem.parentNode&&(e.elem[e.prop]=e.now)}},x.each(["toggle","show","hide"],function(e,t){var n=x.fn[t];x.fn[t]=function(e,r,i){return null==e||"boolean"==typeof e?n.apply(this,arguments):this.animate(ir(t,!0),e,r,i)}}),x.fn.extend({fadeTo:function(e,t,n,r){return this.filter(nn).css("opacity",0).show().end().animate({opacity:t},e,n,r)},animate:function(e,t,n,r){var i=x.isEmptyObject(e),o=x.speed(t,n,r),a=function(){var t=er(this,x.extend({},e),o);(i||x._data(this,"finish"))&&t.stop(!0)};return a.finish=a,i||o.qu
 eue===!1?this.each(a):this.queue(o.queue,a)},stop:function(e,n,r){var i=function(e){var t=e.stop;delete e.stop,t(r)};return"string"!=typeof e&&(r=n,n=e,e=t),n&&e!==!1&&this.queue(e||"fx",[]),this.each(function(){var t=!0,n=null!=e&&e+"queueHooks",o=x.timers,a=x._data(this);if(n)a[n]&&a[n].stop&&i(a[n]);else for(n in a)a[n]&&a[n].stop&&Jn.test(n)&&i(a[n]);for(n=o.length;n--;)o[n].elem!==this||null!=e&&o[n].queue!==e||(o[n].anim.stop(r),t=!1,o.splice(n,1));(t||!r)&&x.dequeue(this,e)})},finish:function(e){return e!==!1&&(e=e||"fx"),this.each(function(){var t,n=x._data(this),r=n[e+"queue"],i=n[e+"queueHooks"],o=x.timers,a=r?r.length:0;for(n.finish=!0,x.queue(this,e,[]),i&&i.stop&&i.stop.call(this,!0),t=o.length;t--;)o[t].elem===this&&o[t].queue===e&&(o[t].anim.stop(!0),o.splice(t,1));for(t=0;a>t;t++)r[t]&&r[t].finish&&r[t].finish.call(this);delete n.finish})}});function ir(e,t){var n,r={height:e},i=0;for(t=t?1:0;4>i;i+=2-t)n=Zt[i],r["marg

<TRUNCATED>


[25/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
GEODE-608 - Adding licence headers to gemfire-site
GEODE-652 - Removing compiled website pages from gemfire-site
          - Added gitignore for gemfire-site


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

Branch: refs/heads/feature/GEODE-14
Commit: 7cf253e86f60fa28af25c33b5847cd6d143ddac8
Parents: 4e4dd7c
Author: William Markito <wm...@pivotal.io>
Authored: Sat Dec 12 12:02:03 2015 -0800
Committer: William Markito <wm...@pivotal.io>
Committed: Sat Dec 12 12:05:20 2015 -0800

----------------------------------------------------------------------
 gemfire-site/.gitignore                         |    1 +
 .../content/bootstrap/bootstrap.min.css         |    9 -
 gemfire-site/content/community/index.html       |  629 -------
 .../content/css/bootflat-extensions.css         |  356 ----
 gemfire-site/content/css/bootflat-square.css    |   69 -
 gemfire-site/content/css/bootflat.css           | 1559 ------------------
 gemfire-site/content/css/font-awesome.min.css   |  405 -----
 gemfire-site/content/css/geode-site.css         | 1554 -----------------
 gemfire-site/content/css/usergrid-site.css      | 1554 -----------------
 gemfire-site/content/favicon.ico                |  Bin 20805 -> 0 bytes
 gemfire-site/content/font/FontAwesome.otf       |  Bin 61896 -> 0 bytes
 .../content/font/fontawesome-webfont-eot.eot    |  Bin 37405 -> 0 bytes
 .../content/font/fontawesome-webfont-svg.svg    |  399 -----
 .../content/font/fontawesome-webfont-ttf.ttf    |  Bin 79076 -> 0 bytes
 .../content/font/fontawesome-webfont-woff.woff  |  Bin 43572 -> 0 bytes
 gemfire-site/content/img/apache_geode_logo.png  |  Bin 23616 -> 0 bytes
 .../content/img/apache_geode_logo_white.png     |  Bin 22695 -> 0 bytes
 .../img/apache_geode_logo_white_small.png       |  Bin 52948 -> 0 bytes
 gemfire-site/content/img/check_flat/default.png |  Bin 25851 -> 0 bytes
 gemfire-site/content/img/egg-logo.png           |  Bin 9938 -> 0 bytes
 gemfire-site/content/img/github.png             |  Bin 8936 -> 0 bytes
 gemfire-site/content/index.html                 |  295 ----
 gemfire-site/content/js/bootstrap.min.js        |    8 -
 gemfire-site/content/js/head.js                 |  708 --------
 gemfire-site/content/js/html5shiv.js            |    8 -
 gemfire-site/content/js/jquery-1.10.1.min.js    |    6 -
 gemfire-site/content/js/jquery.icheck.js        |  397 -----
 gemfire-site/content/js/respond.min.js          |    6 -
 gemfire-site/content/js/usergrid-site.js        |   50 -
 gemfire-site/content/releases/index.html        |  239 ---
 gemfire-site/content/static/github-btn.html     |    2 -
 gemfire-site/website/Rules                      |   19 +-
 gemfire-site/website/build.sh                   |   17 +
 .../website/content/community/index.html        |   16 +
 gemfire-site/website/content/css/geode-site.css |   16 +
 gemfire-site/website/content/index.html         |   16 +
 .../website/content/js/bootstrap.min.js         |    2 +-
 .../website/content/js/usergrid-site.js         |   20 +-
 .../website/content/releases/index.html         |   17 +
 gemfire-site/website/layouts/header.html        |   16 +
 gemfire-site/website/lib/default.rb             |   33 +-
 gemfire-site/website/lib/helpers_.rb            |   16 +
 gemfire-site/website/lib/pandoc.template        |    2 +-
 gemfire-site/website/nanoc.yaml                 |   17 +
 gemfire-site/website/run.sh                     |   17 +
 gemfire-site/website/utilities/map-markers.rb   |   17 +
 .../website/utilities/snapshot-apigee.rb        |   17 +
 gemfire-site/website/utilities/usergrid.csv     |  290 ----
 gradle/rat.gradle                               |   37 +-
 49 files changed, 272 insertions(+), 8567 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/.gitignore
----------------------------------------------------------------------
diff --git a/gemfire-site/.gitignore b/gemfire-site/.gitignore
new file mode 100644
index 0000000..426f0fc
--- /dev/null
+++ b/gemfire-site/.gitignore
@@ -0,0 +1 @@
+/content/*


[12/50] [abbrv] incubator-geode git commit: Revert "Removing TCPConduit's Stub ID class"

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
index 41e5837..a2801c1 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/tcp/TCPConduit.java
@@ -45,7 +45,6 @@ import org.apache.logging.log4j.Logger;
 import com.gemstone.gemfire.CancelCriterion;
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
@@ -64,7 +63,7 @@ import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.logging.log4j.LogMarker;
 
 /** <p>TCPConduit manages a server socket and a collection of connections to
-    other systems.  Connections are identified by DistributedMember IDs.
+    other systems.  Connections are identified by host/port Stubs.
     These types of messages are currently supported:</p><pre>
 
        DistributionMessage - message is delivered to the server's
@@ -176,8 +175,9 @@ public class TCPConduit implements Runnable {
   
   ////////////////// runtime state that is re-initialized on a restart
 
-  /** server socket address */
-  private InetSocketAddress id;
+  /** id is an endpoint Stub representing this server.  It holds the
+      actual port the server is listening on */
+  private Stub id;
 
   protected volatile boolean stopped;
 
@@ -351,7 +351,7 @@ public class TCPConduit implements Runnable {
     try {
       localPort = socket.getLocalPort();
 
-      id = new InetSocketAddress(socket.getInetAddress(), localPort);
+      id = new Stub(socket.getInetAddress(), localPort, 0);
       stopped = false;
       ThreadGroup group =
         LoggingThreadGroup.createThreadGroup("P2P Listener Threads", logger);
@@ -371,12 +371,23 @@ public class TCPConduit implements Runnable {
       }
     }
     catch (IOException io) {
-      String s = "While creating ServerSocket on port " + p;
+      String s = "While creating ServerSocket and Stub on port " + p;
       throw new ConnectionException(s, io);
     }
     this.port = localPort;
   }
   
+  /**
+   * After startup we install the view ID into the conduit stub to avoid
+   * confusion during overlapping shutdown/startup from the same member.
+   * 
+   * @param viewID
+   */
+  public void setVmViewID(int viewID) {
+    this.id.setViewID(viewID);
+  }
+
+
   /** creates the server sockets.  This can be used to recreate the
    *  socket using this.port and this.bindAddress, which must be set
    *  before invoking this method.
@@ -585,7 +596,7 @@ public class TCPConduit implements Runnable {
   public void run() {
     ConnectionTable.threadWantsSharedResources();
     if (logger.isTraceEnabled(LogMarker.DM)) {
-      logger.trace(LogMarker.DM, "Starting P2P Listener on  {}", id);
+      logger.trace(LogMarker.DM, "Starting P2P Listener on  {}", this.getId());
     }
     for(;;) {
       SystemFailure.checkFailure();
@@ -720,7 +731,7 @@ public class TCPConduit implements Runnable {
     } // for
 
     if (logger.isTraceEnabled(LogMarker.DM)) {
-      logger.debug("Stopped P2P Listener on  {}", id);
+      logger.debug("Stopped P2P Listener on  {}", this.getId());
     }
   }
 
@@ -796,7 +807,7 @@ public class TCPConduit implements Runnable {
    * @since 5.1
    */
   public void getThreadOwnedOrderedConnectionState(
-    DistributedMember member,
+    Stub member,
     Map result)
   {
     getConTable().getThreadOwnedOrderedConnectionState(member, result);
@@ -808,7 +819,7 @@ public class TCPConduit implements Runnable {
    * with the key
    * @since 5.1
    */
-  public void waitForThreadOwnedOrderedConnectionState(DistributedMember member, Map channelState)
+  public void waitForThreadOwnedOrderedConnectionState(Stub member, Map channelState)
     throws InterruptedException
   {
     // if (Thread.interrupted()) throw new InterruptedException(); not necessary done in waitForThreadOwnedOrderedConnectionState
@@ -831,12 +842,13 @@ public class TCPConduit implements Runnable {
       msg.setBytesRead(bytesRead);
       msg.setSender(receiver.getRemoteAddress());
       msg.setSharedReceiver(receiver.isSharedResource());
-      directChannel.receive(msg, bytesRead);
+      directChannel.receive(msg, bytesRead, receiver.getRemoteId());
     }
   }
 
-  /** gets the address of this conduit's ServerSocket endpoint */
-  public InetSocketAddress getId() {
+  /** gets the Stub representing this conduit's ServerSocket endpoint.  This
+      is used to generate other stubs containing endpoint information. */
+  public Stub getId() {
     return id;
   }
 
@@ -858,16 +870,21 @@ public class TCPConduit implements Runnable {
   }
 
   
-  /** gets the channel that is used to process non-DistributedMember messages */
+  /** gets the channel that is used to process non-Stub messages */
   public DirectChannel getDirectChannel() {
     return directChannel;
   }
 
+  public InternalDistributedMember getMemberForStub(Stub s, boolean validate) {
+    return membershipManager.getMemberForStub(s, validate);
+  }
+  
   public void setLocalAddr(InternalDistributedMember addr) {
     localAddr = addr;
+    this.id.setViewID(addr.getVmViewId());
   }
   
-  public InternalDistributedMember getLocalAddr() {
+  public InternalDistributedMember getLocalId() {
     return localAddr;
   }
 
@@ -877,6 +894,7 @@ public class TCPConduit implements Runnable {
    * member is in the membership view and the system is not shutting down.
    * 
    * @param memberAddress the IDS associated with the remoteId
+   * @param remoteId the TCPConduit stub for this member
    * @param preserveOrder whether this is an ordered or unordered connection
    * @param retry false if this is the first attempt
    * @param startTime the time this operation started
@@ -884,8 +902,8 @@ public class TCPConduit implements Runnable {
    * @param ackSATimeout the ack-severe-alert-threshold * 1000 for the operation to be transmitted (or zero)
    * @return the connection
    */
-  public Connection getConnection(InternalDistributedMember memberAddress, final boolean preserveOrder, boolean retry, long startTime,
-      long ackTimeout, long ackSATimeout)
+  public Connection getConnection(InternalDistributedMember memberAddress, Stub remoteId, final boolean preserveOrder, boolean retry,
+      long startTime, long ackTimeout, long ackSATimeout)
     throws java.io.IOException, DistributedSystemDisconnectedException
   {
     //final boolean preserveOrder = (processorType == DistributionManager.SERIAL_EXECUTOR )|| (processorType == DistributionManager.PARTITIONED_REGION_EXECUTOR);
@@ -904,7 +922,11 @@ public class TCPConduit implements Runnable {
       // problems.  Tear down the connection so that it gets
       // rebuilt.
       if (retry || conn != null) { // not first time in loop
-        if (!membershipManager.memberExists(memberAddress) || membershipManager.isShunned(memberAddress) || membershipManager.shutdownInProgress()) {
+        // Consult with the membership manager; if member has gone away,
+        // there will not be an entry for this stub.
+        InternalDistributedMember m = this.membershipManager.getMemberForStub(remoteId, true);
+        if (m == null) {
+          // OK, the member left.  Just register an error.
           throw new IOException(LocalizedStrings.TCPConduit_TCPIP_CONNECTION_LOST_AND_MEMBER_IS_NOT_IN_VIEW.toLocalizedString());
         }
         // bug35953: Member is still in view; we MUST NOT give up!
@@ -919,14 +941,15 @@ public class TCPConduit implements Runnable {
         }
         
         // try again after sleep
-        if (!membershipManager.memberExists(memberAddress) || membershipManager.isShunned(memberAddress)) {
+        m = this.membershipManager.getMemberForStub(remoteId, true);
+        if (m == null) {
           // OK, the member left.  Just register an error.
           throw new IOException(LocalizedStrings.TCPConduit_TCPIP_CONNECTION_LOST_AND_MEMBER_IS_NOT_IN_VIEW.toLocalizedString());
         }
         
         // Print a warning (once)
         if (memberInTrouble == null) {
-          memberInTrouble = memberAddress;
+          memberInTrouble = m;
           logger.warn(LocalizedMessage.create(LocalizedStrings.TCPConduit_ATTEMPTING_TCPIP_RECONNECT_TO__0, memberInTrouble));
         }
         else {
@@ -940,8 +963,8 @@ public class TCPConduit implements Runnable {
         if (conn != null) {
           try { 
             if (logger.isDebugEnabled()) {
-              logger.debug("Closing old connection.  conn={} before retrying. memberInTrouble={}",
-                  conn, memberInTrouble);
+              logger.debug("Closing old connection.  conn={} before retrying. remoteID={} memberInTrouble={}",
+                  conn, remoteId,  memberInTrouble);
             }
             conn.closeForReconnect("closing before retrying"); 
           } 
@@ -962,10 +985,10 @@ public class TCPConduit implements Runnable {
         boolean debugRetry = false;
         do {
           retryForOldConnection = false;
-          conn = getConTable().get(memberAddress, preserveOrder, startTime, ackTimeout, ackSATimeout);
+          conn = getConTable().get(remoteId, preserveOrder, startTime, ackTimeout, ackSATimeout);
           if (conn == null) {
             // conduit may be closed - otherwise an ioexception would be thrown
-            problem = new IOException(LocalizedStrings.TCPConduit_UNABLE_TO_RECONNECT_TO_SERVER_POSSIBLE_SHUTDOWN_0.toLocalizedString(memberAddress));
+            problem = new IOException(LocalizedStrings.TCPConduit_UNABLE_TO_RECONNECT_TO_SERVER_POSSIBLE_SHUTDOWN_0.toLocalizedString(remoteId));
           } else if (conn.isClosing() || !conn.getRemoteAddress().equals(memberAddress)) {
             if (logger.isDebugEnabled()) {
               logger.debug("Got an old connection for {}: {}@{}", memberAddress, conn, conn.hashCode());
@@ -1004,14 +1027,15 @@ public class TCPConduit implements Runnable {
 
       if (problem != null) {
         // Some problems are not recoverable; check and error out early.
-        if (!membershipManager.memberExists(memberAddress) || membershipManager.isShunned(memberAddress)) { // left the view
+        InternalDistributedMember m = this.membershipManager.getMemberForStub(remoteId, true);
+        if (m == null) { // left the view
           // Bracket our original warning
           if (memberInTrouble != null) {
             // make this msg info to bracket warning
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.TCPConduit_ENDING_RECONNECT_ATTEMPT_BECAUSE_0_HAS_DISAPPEARED, memberInTrouble));
           }
-          throw new IOException(LocalizedStrings.TCPConduit_PEER_HAS_DISAPPEARED_FROM_VIEW.toLocalizedString(memberAddress));
+          throw new IOException(LocalizedStrings.TCPConduit_PEER_HAS_DISAPPEARED_FROM_VIEW.toLocalizedString(remoteId));
         } // left the view
 
         if (membershipManager.shutdownInProgress()) { // shutdown in progress
@@ -1030,12 +1054,12 @@ public class TCPConduit implements Runnable {
         if (memberInTrouble == null) {
           logger.warn(LocalizedMessage.create(
           LocalizedStrings.TCPConduit_ERROR_SENDING_MESSAGE_TO_0_WILL_REATTEMPT_1,
-          new Object[] {memberAddress, problem}));
-          memberInTrouble = memberAddress;
+          new Object[] {m, problem}));
+          memberInTrouble = m;
         }
         else {
           if (logger.isDebugEnabled()) {
-            logger.debug("Error sending message to {}", memberAddress, problem);
+            logger.debug("Error sending message to {}", m, problem);
           }
         }
 
@@ -1049,7 +1073,7 @@ public class TCPConduit implements Runnable {
             throw (IOException)problem;
           }
           else {
-            IOException ioe = new IOException( LocalizedStrings.TCPConduit_PROBLEM_CONNECTING_TO_0.toLocalizedString(memberAddress));
+            IOException ioe = new IOException( LocalizedStrings.TCPConduit_PROBLEM_CONNECTING_TO_0.toLocalizedString(remoteId));
             ioe.initCause(problem);
             throw ioe;
           }
@@ -1065,8 +1089,8 @@ public class TCPConduit implements Runnable {
             LocalizedStrings.TCPConduit_SUCCESSFULLY_RECONNECTED_TO_MEMBER_0,
             memberInTrouble));
         if (logger.isTraceEnabled()) {
-          logger.trace("new connection is {} memberAddress={}", conn, memberAddress);
-        }
+          logger.trace("new connection is {} remoteId={} memberAddress={}", conn, remoteId, memberAddress);
+      }
       }
       return conn;
       }
@@ -1078,6 +1102,180 @@ public class TCPConduit implements Runnable {
     } // for(;;)
   }
 
+//   /**
+//    * Send a message.
+//    * @return the connection used to send the message
+//    * @throws IOException if peer departed view or shutdown in progress
+//    */
+//   private Connection send(Stub remoteId, ByteBuffer bb, boolean preserveOrder, DistributionMessage msg)
+//     throws java.io.IOException
+//   {
+//     if (stopped) {
+//       throw new ConduitStoppedException("The conduit is stopped");
+//     }
+
+//     if (!QUIET) {
+//       LogWriterI18n l = getLogger();
+//       if (l.finerEnabled()) {
+//         l.finer(id.toString() + " sending " + bb
+//                 + " to " + remoteId);
+//       }
+//     }
+
+//     Connection conn = null;
+//     InternalDistributedMember memberInTrouble = null;
+//     for (;;) {
+//       // If this is the second time through this loop, we had
+//       // problems.  Tear down the connection so that it gets
+//       // rebuilt.
+//       if (conn != null) { // not first time in loop
+//         // Consult with the membership manager; if member has gone away,
+//         // there will not be an entry for this stub.
+//         InternalDistributedMember m = membershipManager.getMemberForStub(remoteId);
+//         if (m == null) {
+//           // OK, the member left.  Just register an error.
+//           throw new IOException("TCP/IP connection lost and member no longer in view");
+//         }
+//         // bug35953: Member is still in view; we MUST NOT give up!
+        
+//         // Pause just a tiny bit...
+//         try {
+//           Thread.sleep(5000);
+//         }
+//         catch (InterruptedException e) {
+//           Thread.currentThread().interrupt();
+//           if (membershipManager.shutdownInProgress()) { // shutdown in progress
+//             // Bracket our original warning
+//             if (memberInTrouble != null) {
+//               logger.info("Ending retry attempt because shutdown has started.");
+//             }
+//             throw new IOException("Abandoned because shutdown is in progress");
+//           } // shutdown in progress
+          
+//           // Strange random interrupt intercepted?
+//           logger.warning("Thread has been interrupted but no shutdown in progress", e);
+//           throw new DistributedSystemDisconnectedException(e);
+//         }
+        
+//         // Print a warning (once)
+//         if (memberInTrouble == null) {
+//           memberInTrouble = m;
+//           getLogger().warning("Attempting TCP/IP reconnect to " + memberInTrouble); 
+//         }
+//         else {
+//           getLogger().fine("Attempting TCP/IP reconnect to " + memberInTrouble);
+//         }
+        
+//         // Close the connection (it will get rebuilt later).
+//         this.stats.incReconnectAttempts();
+//         try { 
+//           conn.closeForReconnect("closing before retrying"); 
+//           } 
+//         catch (CancelException ex) {
+//           // In general we ignore close problems, but if the system
+//           // is shutting down, we should just quit.
+//           throw ex;
+//         }
+//         catch (Exception ex) {
+//           }
+//       } // not first time in loop
+      
+//       // Do the send
+//       Exception problem = null;
+//       try {
+//         // Get (or regenerate) the connection
+//         // bug36202: this could generate a ConnectionException, so it
+//         // must be caught and retried
+//         conn = getConTable().get(remoteId, preserveOrder);
+//         //      getLogger().info ("connections returned " + conn);
+//         if (conn == null) {
+//           // conduit may be closed - otherwise an ioexception would be thrown
+//           throw new IOException("Unable to reconnect to server; possible shutdown: " 
+//               + remoteId);
+//         }
+
+//         conn.sendPreserialized(bb, msg);
+//       }
+//       catch (ConnectionException e) {
+//         // Race condition between acquiring the connection and attempting
+//         // to use it: another thread closed it.
+//         problem = e;
+//       }
+//       catch (IOException e) {
+//         problem = e;
+//       }
+
+//       if (problem != null) {
+//         // Some problems are not recoverable; check an error out early.
+//         InternalDistributedMember m = membershipManager.getMemberForStub(remoteId);
+//         if (m == null) { // left the view
+//           // Bracket our original warning
+//           if (memberInTrouble != null) {
+//             logger.info("Ending retry attempt because " + memberInTrouble 
+//                 + " has disappeared.");
+//           }
+//           throw new IOException("Peer has disappeared from view");
+//         } // left the view
+        
+//         if (membershipManager.shutdownInProgress()) { // shutdown in progress
+//           // Bracket our original warning
+//           if (memberInTrouble != null) {
+//             logger.info("Ending retry attempt because shutdown has started.");
+//           }
+//           throw new IOException("Abandoned because shutdown is in progress");
+//         } // shutdown in progress
+        
+//         if (endpointRemoved(remoteId)) { // endpoint removed
+//           // TODO what does this mean?
+//           // Bracket our original warning
+//           if (memberInTrouble != null) {
+//             logger.info("Ending retry attempt because " + memberInTrouble 
+//                 + " has lost its endpoint.");
+//           }
+//           throw new IOException("Endpoint was removed");
+//         } // endpoint removed
+        
+//         // Log the warning.  We wait until now, because we want
+//         // to have m defined for a nice message...
+//         if (memberInTrouble == null) {
+//           logger.warning(
+//               "Error sending message to " + m + " (will reattempt): " 
+//                   + problem.toString(), 
+//               logger.finerEnabled() ? problem : null);
+//           memberInTrouble = m;
+//         }
+//         else {
+//           logger.fine("Error sending message to " + m, problem);
+//         }
+        
+//         // Retry the operation (indefinitely)
+//         continue;
+//       } // problem != null
+//       // Success!
+      
+//       // Make sure our logging is bracketed if there was a problem
+//       if (memberInTrouble != null) {
+//         logger.info("Successfully reestablished connection to server " 
+//             + memberInTrouble);
+//       }
+//       return conn;
+//     } // while retry
+//   }
+
+//   /**
+//    * Sends an already serialized message in a byte buffer
+//    * to the given endpoint. Waits for the send to complete
+//    * before returning.
+//    * @return the connection used to send the message
+//    */
+//   public Connection sendSync(Stub remoteId, ByteBuffer bb, int processorType, DistributionMessage msg)
+//     throws java.io.IOException
+//   {
+//     return send(remoteId, bb,
+//                 processorType == DistributionManager.SERIAL_EXECUTOR,
+//                 msg);
+//   }
+
   @Override
   public String toString() {
     return "" + id;
@@ -1103,22 +1301,22 @@ public class TCPConduit implements Runnable {
     return directChannel.getDM();
   }
   /**
-   * Closes any connections used to communicate with the given member
+   * Closes any connections used to communicate with the given stub
    */
-  public void removeEndpoint(DistributedMember mbr, String reason) {
-    removeEndpoint(mbr, reason, true);
+  public void removeEndpoint(Stub stub, String reason) {
+    removeEndpoint(stub, reason, true);
   }
   
-  public void removeEndpoint(DistributedMember mbr, String reason, boolean notifyDisconnect) {
+  public void removeEndpoint(Stub stub, String reason, boolean notifyDisconnect) {
     ConnectionTable ct = this.conTable;
     if (ct == null) {
       return;
     }
-    ct.removeEndpoint(mbr, reason, notifyDisconnect);
+    ct.removeEndpoint(stub, reason, notifyDisconnect);
   }
   
   /** check to see if there are still any receiver threads for the given end-point */
-  public boolean hasReceiversFor(DistributedMember endPoint) {
+  public boolean hasReceiversFor(Stub endPoint) {
     ConnectionTable ct = this.conTable;
     return (ct != null) && ct.hasReceiversFor(endPoint);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
index 773ef38..1f411bb 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/DistributionManagerDUnitTest.java
@@ -38,6 +38,7 @@ 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.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
@@ -45,10 +46,13 @@ import com.gemstone.gemfire.distributed.internal.membership.NetView;
 import com.gemstone.gemfire.distributed.internal.membership.gms.MembershipManagerHelper;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
+import com.gemstone.gemfire.internal.AvailablePort;
 import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.internal.tcp.Stub;
 
 import dunit.DistributedTestCase;
 import dunit.Host;
+import dunit.SerializableCallable;
 import dunit.SerializableRunnable;
 import dunit.VM;
 
@@ -188,7 +192,7 @@ public class DistributionManagerDUnitTest extends DistributedTestCase {
       sys.getLogWriter().info("<ExpectedException action=add>attempt to add old member</ExpectedException>");
       sys.getLogWriter().info("<ExpectedException action=add>Removing shunned GemFire node</ExpectedException>");
       try {
-        boolean accepted = mgr.addSurpriseMember(mbr);
+        boolean accepted = mgr.addSurpriseMember(mbr, new Stub());
         Assert.assertTrue("member with old ID was not rejected (bug #44566)", !accepted);
       } finally {
         sys.getLogWriter().info("<ExpectedException action=remove>attempt to add old member</ExpectedException>");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
index ddbda0b..44e1b46 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/membership/gms/mgr/GMSMembershipManagerJUnitTest.java
@@ -16,18 +16,8 @@
  */
 package com.gemstone.gemfire.distributed.internal.membership.gms.mgr;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.isA;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.reset;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -46,6 +36,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.internal.AdminMessageType;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.DMStats;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
@@ -53,6 +44,7 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfigImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionManager;
 import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.HighPriorityAckedMessage;
+import com.gemstone.gemfire.distributed.internal.HighPriorityDistributionMessage;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.MembershipListener;
 import com.gemstone.gemfire.distributed.internal.ReplyProcessor21;
@@ -68,11 +60,19 @@ import com.gemstone.gemfire.distributed.internal.membership.gms.SuspectMember;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Authenticator;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.HealthMonitor;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.JoinLeave;
+import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Manager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.interfaces.Messenger;
+import com.gemstone.gemfire.distributed.internal.membership.gms.membership.GMSJoinLeave;
+import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.gms.mgr.GMSMembershipManager.StartupEvent;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.admin.remote.AdminRequest;
+import com.gemstone.gemfire.internal.admin.remote.AdminResponse;
 import com.gemstone.gemfire.internal.admin.remote.AlertListenerMessage;
 import com.gemstone.gemfire.internal.admin.remote.RemoteTransportConfig;
 import com.gemstone.gemfire.internal.tcp.ConnectExceptions;
+import com.gemstone.gemfire.internal.tcp.Stub;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -295,6 +295,13 @@ public class GMSMembershipManagerJUnitTest {
     suspectMember = mockMembers[1];
     manager.handleOrDeferSuspect(new SuspectMember(mockMembers[0], suspectMember, "testing"));
     verify(listener).memberSuspect(suspectMember, mockMembers[0], "testing");
+    
+    InternalDistributedMember mbr = manager.getMemberForStub(new Stub(myMemberId.getInetAddress(), 2033, 20), false);
+    assertTrue(mbr == null);
+    myMemberId.setDirectChannelPort(2033);
+    mbr = manager.getMemberForStub(new Stub(myMemberId.getInetAddress(), 2033, 20), false);
+    assertTrue(mbr != null);
+    assertEquals(mbr, myMemberId);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/507f2f3a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
index ffd5092..78c462f 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/tcp/ConnectionJUnitTest.java
@@ -22,7 +22,6 @@ import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.verify;
 
 import java.io.InputStream;
-import java.net.InetSocketAddress;
 import java.net.Socket;
 
 import org.junit.Test;
@@ -62,7 +61,7 @@ public class ConnectionJUnitTest {
     when(stopper.cancelInProgress()).thenReturn(null);
     when(conduit.getCancelCriterion()).thenReturn(stopper);
 
-    when(conduit.getId()).thenReturn(new InetSocketAddress(SocketCreator.getLocalHost(), 10337));
+    when(conduit.getId()).thenReturn(new Stub(SocketCreator.getLocalHost(), 10337, 1));
     
     // NIO can't be mocked because SocketChannel has a final method that
     // is used by Connection - configureBlocking


[24/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/bootstrap/bootstrap.min.css
----------------------------------------------------------------------
diff --git a/gemfire-site/content/bootstrap/bootstrap.min.css b/gemfire-site/content/bootstrap/bootstrap.min.css
deleted file mode 100644
index 93c646f..0000000
--- a/gemfire-site/content/bootstrap/bootstrap.min.css
+++ /dev/null
@@ -1,9 +0,0 @@
-/*!
- * Bootstrap v3.0.0
- *
- * Copyright 2013 Twitter, Inc
- * Licensed under the Apache License v2.0
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Designed and built with all the love in the world by @mdo and @fat.
- *//*! normalize.css v2.1.0 | MIT License | git.io/normalize */article,aside,details,figcaption,figure,footer,header,hgroup,main,nav,section,summary{display:block}audio,canvas,video{display:inline-block}audio:not([controls]){display:none;height:0}[hidden]{display:none}html{font-family:sans-serif;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}body{margin:0}a:focus{outline:thin dotted}a:active,a:hover{outline:0}h1{margin:.67em 0;font-size:2em}abbr[title]{border-bottom:1px dotted}b,strong{font-weight:bold}dfn{font-style:italic}hr{height:0;-moz-box-sizing:content-box;box-sizing:content-box}mark{color:#000;background:#ff0}code,kbd,pre,samp{font-family:monospace,serif;font-size:1em}pre{white-space:pre-wrap}q{quotes:"\201C" "\201D" "\2018" "\2019"}small{font-size:80%}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{border:0}svg:not(:root){overflow:hidden}figure{margin:0}fieldset{padding:.35em .625em .75em;margin:0 
 2px;border:1px solid #c0c0c0}legend{padding:0;border:0}button,input,select,textarea{margin:0;font-family:inherit;font-size:100%}button,input{line-height:normal}button,select{text-transform:none}button,html input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}button[disabled],html input[disabled]{cursor:default}input[type="checkbox"],input[type="radio"]{padding:0;box-sizing:border-box}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-cancel-button,input[type="search"]::-webkit-search-decoration{-webkit-appearance:none}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}textarea{overflow:auto;vertical-align:top}table{border-collapse:collapse;border-spacing:0}@media print{*{color:#000!important;text-shadow:none!important;background:transparent!important;box-shadow:none!important}a,a:visited{text-decorati
 on:underline}a[href]:after{content:" (" attr(href) ")"}abbr[title]:after{content:" (" attr(title) ")"}.ir a:after,a[href^="javascript:"]:after,a[href^="#"]:after{content:""}pre,blockquote{border:1px solid #999;page-break-inside:avoid}thead{display:table-header-group}tr,img{page-break-inside:avoid}img{max-width:100%!important}@page{margin:2cm .5cm}p,h2,h3{orphans:3;widows:3}h2,h3{page-break-after:avoid}.navbar{display:none}.table td,.table th{background-color:#fff!important}.btn>.caret,.dropup>.btn>.caret{border-top-color:#000!important}.label{border:1px solid #000}.table{border-collapse:collapse!important}.table-bordered th,.table-bordered td{border:1px solid #ddd!important}}*,*:before,*:after{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}html{font-size:62.5%;-webkit-tap-highlight-color:rgba(0,0,0,0)}body{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:1.428571429;color:#333;background-color:#fff}input,button,select
 ,textarea{font-family:inherit;font-size:inherit;line-height:inherit}button,input,select[multiple],textarea{background-image:none}a{color:#428bca;text-decoration:none}a:hover,a:focus{color:#2a6496;text-decoration:underline}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}img{vertical-align:middle}.img-responsive{display:block;height:auto;max-width:100%}.img-rounded{border-radius:6px}.img-thumbnail{display:inline-block;height:auto;max-width:100%;padding:4px;line-height:1.428571429;background-color:#fff;border:1px solid #ddd;border-radius:4px;-webkit-transition:all .2s ease-in-out;transition:all .2s ease-in-out}.img-circle{border-radius:50%}hr{margin-top:20px;margin-bottom:20px;border:0;border-top:1px solid #eee}.sr-only{position:absolute;width:1px;height:1px;padding:0;margin:-1px;overflow:hidden;clip:rect(0 0 0 0);border:0}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:16.099999999999998px;font-weight:200;line-height:1.4}@media(min-w
 idth:768px){.lead{font-size:21px}}small{font-size:85%}cite{font-style:normal}.text-muted{color:#999}.text-primary{color:#428bca}.text-warning{color:#c09853}.text-danger{color:#b94a48}.text-success{color:#468847}.text-info{color:#3a87ad}.text-left{text-align:left}.text-right{text-align:right}.text-center{text-align:center}h1,h2,h3,h4,h5,h6,.h1,.h2,.h3,.h4,.h5,.h6{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-weight:500;line-height:1.1}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small,.h1 small,.h2 small,.h3 small,.h4 small,.h5 small,.h6 small{font-weight:normal;line-height:1;color:#999}h1,h2,h3{margin-top:20px;margin-bottom:10px}h1 small,h2 small,h3 small{font-size:65%}h4,h5,h6{margin-top:10px;margin-bottom:10px}h4 small,h5 small,h6 small{font-size:75%}h1,.h1{font-size:36px}h2,.h2{font-size:30px}h3,.h3{font-size:24px}h4,.h4{font-size:18px}h5,.h5{font-size:14px}h6,.h6{font-size:12px}.page-header{padding-bottom:9px;margin:40px 0 20px;border-bottom:1px solid #eee}ul,o
 l{margin-top:0;margin-bottom:10px}ul ul,ol ul,ul ol,ol ol{margin-bottom:0}.list-unstyled{padding-left:0;list-style:none}.list-inline{padding-left:0;list-style:none}.list-inline>li{display:inline-block;padding-right:5px;padding-left:5px}dl{margin-bottom:20px}dt,dd{line-height:1.428571429}dt{font-weight:bold}dd{margin-left:0}@media(min-width:768px){.dl-horizontal dt{float:left;width:160px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:180px}.dl-horizontal dd:before,.dl-horizontal dd:after{display:table;content:" "}.dl-horizontal dd:after{clear:both}.dl-horizontal dd:before,.dl-horizontal dd:after{display:table;content:" "}.dl-horizontal dd:after{clear:both}}abbr[title],abbr[data-original-title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:10px 20px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{font-size:17.5px;font-weight:300;line-height
 :1.25}blockquote p:last-child{margin-bottom:0}blockquote small{display:block;line-height:1.428571429;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:1.428571429}code,pre{font-family:Monaco,Menlo,Consolas,"Courier New",monospace}code{padding:2px 4px;font-size:90%;color:#c7254e;white-space:nowrap;background-color:#f9f2f4;border-radius:4px}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:1.428571429;color:#333;word-break:break-all;word-wrap:break-word;background-color:#f5f5f5;border:1px solid #ccc;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre co
 de{padding:0;font-size:inherit;color:inherit;white-space:pre-wrap;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}.container{padding-right:15px;padding-left:15px;margin-right:auto;margin-left:auto}.container:before,.container:after{display:table;content:" "}.container:after{clear:both}.container:before,.container:after{display:table;content:" "}.container:after{clear:both}.row{margin-right:-15px;margin-left:-15px}.row:before,.row:after{display:table;content:" "}.row:after{clear:both}.row:before,.row:after{display:table;content:" "}.row:after{clear:both}.col-xs-1,.col-xs-2,.col-xs-3,.col-xs-4,.col-xs-5,.col-xs-6,.col-xs-7,.col-xs-8,.col-xs-9,.col-xs-10,.col-xs-11,.col-xs-12,.col-sm-1,.col-sm-2,.col-sm-3,.col-sm-4,.col-sm-5,.col-sm-6,.col-sm-7,.col-sm-8,.col-sm-9,.col-sm-10,.col-sm-11,.col-sm-12,.col-md-1,.col-md-2,.col-md-3,.col-md-4,.col-md-5,.col-md-6,.col-md-7,.col-md-8,.col-md-9,.col-md-10,.col-md-11,.col-md-12,.col-lg-1,.col-lg-2,.col-lg-
 3,.col-lg-4,.col-lg-5,.col-lg-6,.col-lg-7,.col-lg-8,.col-lg-9,.col-lg-10,.col-lg-11,.col-lg-12{position:relative;min-height:1px;padding-right:15px;padding-left:15px}.col-xs-1,.col-xs-2,.col-xs-3,.col-xs-4,.col-xs-5,.col-xs-6,.col-xs-7,.col-xs-8,.col-xs-9,.col-xs-10,.col-xs-11{float:left}.col-xs-1{width:8.333333333333332%}.col-xs-2{width:16.666666666666664%}.col-xs-3{width:25%}.col-xs-4{width:33.33333333333333%}.col-xs-5{width:41.66666666666667%}.col-xs-6{width:50%}.col-xs-7{width:58.333333333333336%}.col-xs-8{width:66.66666666666666%}.col-xs-9{width:75%}.col-xs-10{width:83.33333333333334%}.col-xs-11{width:91.66666666666666%}.col-xs-12{width:100%}@media(min-width:768px){.container{max-width:750px}.col-sm-1,.col-sm-2,.col-sm-3,.col-sm-4,.col-sm-5,.col-sm-6,.col-sm-7,.col-sm-8,.col-sm-9,.col-sm-10,.col-sm-11{float:left}.col-sm-1{width:8.333333333333332%}.col-sm-2{width:16.666666666666664%}.col-sm-3{width:25%}.col-sm-4{width:33.33333333333333%}.col-sm-5{width:41.66666666666667%}.col-sm-
 6{width:50%}.col-sm-7{width:58.333333333333336%}.col-sm-8{width:66.66666666666666%}.col-sm-9{width:75%}.col-sm-10{width:83.33333333333334%}.col-sm-11{width:91.66666666666666%}.col-sm-12{width:100%}.col-sm-push-1{left:8.333333333333332%}.col-sm-push-2{left:16.666666666666664%}.col-sm-push-3{left:25%}.col-sm-push-4{left:33.33333333333333%}.col-sm-push-5{left:41.66666666666667%}.col-sm-push-6{left:50%}.col-sm-push-7{left:58.333333333333336%}.col-sm-push-8{left:66.66666666666666%}.col-sm-push-9{left:75%}.col-sm-push-10{left:83.33333333333334%}.col-sm-push-11{left:91.66666666666666%}.col-sm-pull-1{right:8.333333333333332%}.col-sm-pull-2{right:16.666666666666664%}.col-sm-pull-3{right:25%}.col-sm-pull-4{right:33.33333333333333%}.col-sm-pull-5{right:41.66666666666667%}.col-sm-pull-6{right:50%}.col-sm-pull-7{right:58.333333333333336%}.col-sm-pull-8{right:66.66666666666666%}.col-sm-pull-9{right:75%}.col-sm-pull-10{right:83.33333333333334%}.col-sm-pull-11{right:91.66666666666666%}.col-sm-offse
 t-1{margin-left:8.333333333333332%}.col-sm-offset-2{margin-left:16.666666666666664%}.col-sm-offset-3{margin-left:25%}.col-sm-offset-4{margin-left:33.33333333333333%}.col-sm-offset-5{margin-left:41.66666666666667%}.col-sm-offset-6{margin-left:50%}.col-sm-offset-7{margin-left:58.333333333333336%}.col-sm-offset-8{margin-left:66.66666666666666%}.col-sm-offset-9{margin-left:75%}.col-sm-offset-10{margin-left:83.33333333333334%}.col-sm-offset-11{margin-left:91.66666666666666%}}@media(min-width:992px){.container{max-width:970px}.col-md-1,.col-md-2,.col-md-3,.col-md-4,.col-md-5,.col-md-6,.col-md-7,.col-md-8,.col-md-9,.col-md-10,.col-md-11{float:left}.col-md-1{width:8.333333333333332%}.col-md-2{width:16.666666666666664%}.col-md-3{width:25%}.col-md-4{width:33.33333333333333%}.col-md-5{width:41.66666666666667%}.col-md-6{width:50%}.col-md-7{width:58.333333333333336%}.col-md-8{width:66.66666666666666%}.col-md-9{width:75%}.col-md-10{width:83.33333333333334%}.col-md-11{width:91.66666666666666%}.col
 -md-12{width:100%}.col-md-push-0{left:auto}.col-md-push-1{left:8.333333333333332%}.col-md-push-2{left:16.666666666666664%}.col-md-push-3{left:25%}.col-md-push-4{left:33.33333333333333%}.col-md-push-5{left:41.66666666666667%}.col-md-push-6{left:50%}.col-md-push-7{left:58.333333333333336%}.col-md-push-8{left:66.66666666666666%}.col-md-push-9{left:75%}.col-md-push-10{left:83.33333333333334%}.col-md-push-11{left:91.66666666666666%}.col-md-pull-0{right:auto}.col-md-pull-1{right:8.333333333333332%}.col-md-pull-2{right:16.666666666666664%}.col-md-pull-3{right:25%}.col-md-pull-4{right:33.33333333333333%}.col-md-pull-5{right:41.66666666666667%}.col-md-pull-6{right:50%}.col-md-pull-7{right:58.333333333333336%}.col-md-pull-8{right:66.66666666666666%}.col-md-pull-9{right:75%}.col-md-pull-10{right:83.33333333333334%}.col-md-pull-11{right:91.66666666666666%}.col-md-offset-0{margin-left:0}.col-md-offset-1{margin-left:8.333333333333332%}.col-md-offset-2{margin-left:16.666666666666664%}.col-md-offse
 t-3{margin-left:25%}.col-md-offset-4{margin-left:33.33333333333333%}.col-md-offset-5{margin-left:41.66666666666667%}.col-md-offset-6{margin-left:50%}.col-md-offset-7{margin-left:58.333333333333336%}.col-md-offset-8{margin-left:66.66666666666666%}.col-md-offset-9{margin-left:75%}.col-md-offset-10{margin-left:83.33333333333334%}.col-md-offset-11{margin-left:91.66666666666666%}}@media(min-width:1200px){.container{max-width:1170px}.col-lg-1,.col-lg-2,.col-lg-3,.col-lg-4,.col-lg-5,.col-lg-6,.col-lg-7,.col-lg-8,.col-lg-9,.col-lg-10,.col-lg-11{float:left}.col-lg-1{width:8.333333333333332%}.col-lg-2{width:16.666666666666664%}.col-lg-3{width:25%}.col-lg-4{width:33.33333333333333%}.col-lg-5{width:41.66666666666667%}.col-lg-6{width:50%}.col-lg-7{width:58.333333333333336%}.col-lg-8{width:66.66666666666666%}.col-lg-9{width:75%}.col-lg-10{width:83.33333333333334%}.col-lg-11{width:91.66666666666666%}.col-lg-12{width:100%}.col-lg-push-0{left:auto}.col-lg-push-1{left:8.333333333333332%}.col-lg-push-
 2{left:16.666666666666664%}.col-lg-push-3{left:25%}.col-lg-push-4{left:33.33333333333333%}.col-lg-push-5{left:41.66666666666667%}.col-lg-push-6{left:50%}.col-lg-push-7{left:58.333333333333336%}.col-lg-push-8{left:66.66666666666666%}.col-lg-push-9{left:75%}.col-lg-push-10{left:83.33333333333334%}.col-lg-push-11{left:91.66666666666666%}.col-lg-pull-0{right:auto}.col-lg-pull-1{right:8.333333333333332%}.col-lg-pull-2{right:16.666666666666664%}.col-lg-pull-3{right:25%}.col-lg-pull-4{right:33.33333333333333%}.col-lg-pull-5{right:41.66666666666667%}.col-lg-pull-6{right:50%}.col-lg-pull-7{right:58.333333333333336%}.col-lg-pull-8{right:66.66666666666666%}.col-lg-pull-9{right:75%}.col-lg-pull-10{right:83.33333333333334%}.col-lg-pull-11{right:91.66666666666666%}.col-lg-offset-0{margin-left:0}.col-lg-offset-1{margin-left:8.333333333333332%}.col-lg-offset-2{margin-left:16.666666666666664%}.col-lg-offset-3{margin-left:25%}.col-lg-offset-4{margin-left:33.33333333333333%}.col-lg-offset-5{margin-lef
 t:41.66666666666667%}.col-lg-offset-6{margin-left:50%}.col-lg-offset-7{margin-left:58.333333333333336%}.col-lg-offset-8{margin-left:66.66666666666666%}.col-lg-offset-9{margin-left:75%}.col-lg-offset-10{margin-left:83.33333333333334%}.col-lg-offset-11{margin-left:91.66666666666666%}}table{max-width:100%;background-color:transparent}th{text-align:left}.table{width:100%;margin-bottom:20px}.table thead>tr>th,.table tbody>tr>th,.table tfoot>tr>th,.table thead>tr>td,.table tbody>tr>td,.table tfoot>tr>td{padding:8px;line-height:1.428571429;vertical-align:top;border-top:1px solid #ddd}.table thead>tr>th{vertical-align:bottom;border-bottom:2px solid #ddd}.table caption+thead tr:first-child th,.table colgroup+thead tr:first-child th,.table thead:first-child tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table .table{background-color:#fff}.tabl
 e-condensed thead>tr>th,.table-condensed tbody>tr>th,.table-condensed tfoot>tr>th,.table-condensed thead>tr>td,.table-condensed tbody>tr>td,.table-condensed tfoot>tr>td{padding:5px}.table-bordered{border:1px solid #ddd}.table-bordered>thead>tr>th,.table-bordered>tbody>tr>th,.table-bordered>tfoot>tr>th,.table-bordered>thead>tr>td,.table-bordered>tbody>tr>td,.table-bordered>tfoot>tr>td{border:1px solid #ddd}.table-bordered>thead>tr>th,.table-bordered>thead>tr>td{border-bottom-width:2px}.table-striped>tbody>tr:nth-child(odd)>td,.table-striped>tbody>tr:nth-child(odd)>th{background-color:#f9f9f9}.table-hover>tbody>tr:hover>td,.table-hover>tbody>tr:hover>th{background-color:#f5f5f5}table col[class*="col-"]{display:table-column;float:none}table td[class*="col-"],table th[class*="col-"]{display:table-cell;float:none}.table>thead>tr>td.active,.table>tbody>tr>td.active,.table>tfoot>tr>td.active,.table>thead>tr>th.active,.table>tbody>tr>th.active,.table>tfoot>tr>th.active,.table>thead>tr.activ
 e>td,.table>tbody>tr.active>td,.table>tfoot>tr.active>td,.table>thead>tr.active>th,.table>tbody>tr.active>th,.table>tfoot>tr.active>th{background-color:#f5f5f5}.table>thead>tr>td.success,.table>tbody>tr>td.success,.table>tfoot>tr>td.success,.table>thead>tr>th.success,.table>tbody>tr>th.success,.table>tfoot>tr>th.success,.table>thead>tr.success>td,.table>tbody>tr.success>td,.table>tfoot>tr.success>td,.table>thead>tr.success>th,.table>tbody>tr.success>th,.table>tfoot>tr.success>th{background-color:#dff0d8;border-color:#d6e9c6}.table-hover>tbody>tr>td.success:hover,.table-hover>tbody>tr>th.success:hover,.table-hover>tbody>tr.success:hover>td,.table-hover>tbody>tr.success:hover>th{background-color:#d0e9c6;border-color:#c9e2b3}.table>thead>tr>td.danger,.table>tbody>tr>td.danger,.table>tfoot>tr>td.danger,.table>thead>tr>th.danger,.table>tbody>tr>th.danger,.table>tfoot>tr>th.danger,.table>thead>tr.danger>td,.table>tbody>tr.danger>td,.table>tfoot>tr.danger>td,.table>thead>tr.danger>th,.tabl
 e>tbody>tr.danger>th,.table>tfoot>tr.danger>th{background-color:#f2dede;border-color:#eed3d7}.table-hover>tbody>tr>td.danger:hover,.table-hover>tbody>tr>th.danger:hover,.table-hover>tbody>tr.danger:hover>td,.table-hover>tbody>tr.danger:hover>th{background-color:#ebcccc;border-color:#e6c1c7}.table>thead>tr>td.warning,.table>tbody>tr>td.warning,.table>tfoot>tr>td.warning,.table>thead>tr>th.warning,.table>tbody>tr>th.warning,.table>tfoot>tr>th.warning,.table>thead>tr.warning>td,.table>tbody>tr.warning>td,.table>tfoot>tr.warning>td,.table>thead>tr.warning>th,.table>tbody>tr.warning>th,.table>tfoot>tr.warning>th{background-color:#fcf8e3;border-color:#fbeed5}.table-hover>tbody>tr>td.warning:hover,.table-hover>tbody>tr>th.warning:hover,.table-hover>tbody>tr.warning:hover>td,.table-hover>tbody>tr.warning:hover>th{background-color:#faf2cc;border-color:#f8e5be}@media(max-width:768px){.table-responsive{width:100%;margin-bottom:15px;overflow-x:scroll;overflow-y:hidden;border:1px solid #ddd;-ms-
 overflow-style:-ms-autohiding-scrollbar;-webkit-overflow-scrolling:touch}.table-responsive>.table{margin-bottom:0;background-color:#fff}.table-responsive>.table>thead>tr>th,.table-responsive>.table>tbody>tr>th,.table-responsive>.table>tfoot>tr>th,.table-responsive>.table>thead>tr>td,.table-responsive>.table>tbody>tr>td,.table-responsive>.table>tfoot>tr>td{white-space:nowrap}.table-responsive>.table-bordered{border:0}.table-responsive>.table-bordered>thead>tr>th:first-child,.table-responsive>.table-bordered>tbody>tr>th:first-child,.table-responsive>.table-bordered>tfoot>tr>th:first-child,.table-responsive>.table-bordered>thead>tr>td:first-child,.table-responsive>.table-bordered>tbody>tr>td:first-child,.table-responsive>.table-bordered>tfoot>tr>td:first-child{border-left:0}.table-responsive>.table-bordered>thead>tr>th:last-child,.table-responsive>.table-bordered>tbody>tr>th:last-child,.table-responsive>.table-bordered>tfoot>tr>th:last-child,.table-responsive>.table-bordered>thead>tr>t
 d:last-child,.table-responsive>.table-bordered>tbody>tr>td:last-child,.table-responsive>.table-bordered>tfoot>tr>td:last-child{border-right:0}.table-responsive>.table-bordered>thead>tr:last-child>th,.table-responsive>.table-bordered>tbody>tr:last-child>th,.table-responsive>.table-bordered>tfoot>tr:last-child>th,.table-responsive>.table-bordered>thead>tr:last-child>td,.table-responsive>.table-bordered>tbody>tr:last-child>td,.table-responsive>.table-bordered>tfoot>tr:last-child>td{border-bottom:0}}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:inherit;color:#333;border:0;border-bottom:1px solid #e5e5e5}label{display:inline-block;margin-bottom:5px;font-weight:bold}input[type="search"]{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;line-height:normal}input[type="file"]{display:block}select[multiple],sele
 ct[size]{height:auto}select optgroup{font-family:inherit;font-size:inherit;font-style:inherit}input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}input[type="number"]::-webkit-outer-spin-button,input[type="number"]::-webkit-inner-spin-button{height:auto}output{display:block;padding-top:7px;font-size:14px;line-height:1.428571429;color:#555;vertical-align:middle}.form-control:-moz-placeholder{color:#999}.form-control::-moz-placeholder{color:#999}.form-control:-ms-input-placeholder{color:#999}.form-control::-webkit-input-placeholder{color:#999}.form-control{display:block;width:100%;height:34px;padding:6px 12px;font-size:14px;line-height:1.428571429;color:#555;vertical-align:middle;background-color:#fff;border:1px solid #ccc;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border-color eas
 e-in-out .15s,box-shadow ease-in-out .15s;transition:border-color ease-in-out .15s,box-shadow ease-in-out .15s}.form-control:focus{border-color:#66afe9;outline:0;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(102,175,233,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(102,175,233,0.6)}.form-control[disabled],.form-control[readonly],fieldset[disabled] .form-control{cursor:not-allowed;background-color:#eee}textarea.form-control{height:auto}.form-group{margin-bottom:15px}.radio,.checkbox{display:block;min-height:20px;padding-left:20px;margin-top:10px;margin-bottom:10px;vertical-align:middle}.radio label,.checkbox label{display:inline;margin-bottom:0;font-weight:normal;cursor:pointer}.radio input[type="radio"],.radio-inline input[type="radio"],.checkbox input[type="checkbox"],.checkbox-inline input[type="checkbox"]{float:left;margin-left:-20px}.radio+.radio,.checkbox+.checkbox{margin-top:-5px}.radio-inline,.checkbox-inline{display:inline-block;padding-l
 eft:20px;margin-bottom:0;font-weight:normal;vertical-align:middle;cursor:pointer}.radio-inline+.radio-inline,.checkbox-inline+.checkbox-inline{margin-top:0;margin-left:10px}input[type="radio"][disabled],input[type="checkbox"][disabled],.radio[disabled],.radio-inline[disabled],.checkbox[disabled],.checkbox-inline[disabled],fieldset[disabled] input[type="radio"],fieldset[disabled] input[type="checkbox"],fieldset[disabled] .radio,fieldset[disabled] .radio-inline,fieldset[disabled] .checkbox,fieldset[disabled] .checkbox-inline{cursor:not-allowed}.input-sm{height:30px;padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}select.input-sm{height:30px;line-height:30px}textarea.input-sm{height:auto}.input-lg{height:45px;padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}select.input-lg{height:45px;line-height:45px}textarea.input-lg{height:auto}.has-warning .help-block,.has-warning .control-label{color:#c09853}.has-warning .form-control{border-color:#c09853;-webki
 t-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.has-warning .form-control:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.has-warning .input-group-addon{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.has-error .help-block,.has-error .control-label{color:#b94a48}.has-error .form-control{border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.has-error .form-control:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.has-error .input-group-addon{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.has-success .help-block,.has-success .control-label{color:#468847}.has-success .form-control{border-color:#468847;-webkit-box-shadow:inset 0 1px 1px 
 rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.has-success .form-control:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.has-success .input-group-addon{color:#468847;background-color:#dff0d8;border-color:#468847}.form-control-static{padding-top:7px;margin-bottom:0}.help-block{display:block;margin-top:5px;margin-bottom:10px;color:#737373}@media(min-width:768px){.form-inline .form-group{display:inline-block;margin-bottom:0;vertical-align:middle}.form-inline .form-control{display:inline-block}.form-inline .radio,.form-inline .checkbox{display:inline-block;padding-left:0;margin-top:0;margin-bottom:0}.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:none;margin-left:0}}.form-horizontal .control-label,.form-horizontal .radio,.form-horizontal .checkbox,.form-horizontal .radio-inline,.form-horizontal .checkbox-inline{padding
 -top:7px;margin-top:0;margin-bottom:0}.form-horizontal .form-group{margin-right:-15px;margin-left:-15px}.form-horizontal .form-group:before,.form-horizontal .form-group:after{display:table;content:" "}.form-horizontal .form-group:after{clear:both}.form-horizontal .form-group:before,.form-horizontal .form-group:after{display:table;content:" "}.form-horizontal .form-group:after{clear:both}@media(min-width:768px){.form-horizontal .control-label{text-align:right}}.btn{display:inline-block;padding:6px 12px;margin-bottom:0;font-size:14px;font-weight:normal;line-height:1.428571429;text-align:center;white-space:nowrap;vertical-align:middle;cursor:pointer;border:1px solid transparent;border-radius:4px;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;-o-user-select:none;user-select:none}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn:hover,.btn:focus{color:#333;text-decoration:none}.btn:active,.btn.active{background-ima
 ge:none;outline:0;-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,0.125);box-shadow:inset 0 3px 5px rgba(0,0,0,0.125)}.btn.disabled,.btn[disabled],fieldset[disabled] .btn{pointer-events:none;cursor:not-allowed;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;box-shadow:none}.btn-default{color:#333;background-color:#fff;border-color:#ccc}.btn-default:hover,.btn-default:focus,.btn-default:active,.btn-default.active,.open .dropdown-toggle.btn-default{color:#333;background-color:#ebebeb;border-color:#adadad}.btn-default:active,.btn-default.active,.open .dropdown-toggle.btn-default{background-image:none}.btn-default.disabled,.btn-default[disabled],fieldset[disabled] .btn-default,.btn-default.disabled:hover,.btn-default[disabled]:hover,fieldset[disabled] .btn-default:hover,.btn-default.disabled:focus,.btn-default[disabled]:focus,fieldset[disabled] .btn-default:focus,.btn-default.disabled:active,.btn-default[disabled]:active,fieldset[disabled] .btn-default:active,.btn-default.dis
 abled.active,.btn-default[disabled].active,fieldset[disabled] .btn-default.active{background-color:#fff;border-color:#ccc}.btn-primary{color:#fff;background-color:#428bca;border-color:#357ebd}.btn-primary:hover,.btn-primary:focus,.btn-primary:active,.btn-primary.active,.open .dropdown-toggle.btn-primary{color:#fff;background-color:#3276b1;border-color:#285e8e}.btn-primary:active,.btn-primary.active,.open .dropdown-toggle.btn-primary{background-image:none}.btn-primary.disabled,.btn-primary[disabled],fieldset[disabled] .btn-primary,.btn-primary.disabled:hover,.btn-primary[disabled]:hover,fieldset[disabled] .btn-primary:hover,.btn-primary.disabled:focus,.btn-primary[disabled]:focus,fieldset[disabled] .btn-primary:focus,.btn-primary.disabled:active,.btn-primary[disabled]:active,fieldset[disabled] .btn-primary:active,.btn-primary.disabled.active,.btn-primary[disabled].active,fieldset[disabled] .btn-primary.active{background-color:#428bca;border-color:#357ebd}.btn-warning{color:#fff;backg
 round-color:#f0ad4e;border-color:#eea236}.btn-warning:hover,.btn-warning:focus,.btn-warning:active,.btn-warning.active,.open .dropdown-toggle.btn-warning{color:#fff;background-color:#ed9c28;border-color:#d58512}.btn-warning:active,.btn-warning.active,.open .dropdown-toggle.btn-warning{background-image:none}.btn-warning.disabled,.btn-warning[disabled],fieldset[disabled] .btn-warning,.btn-warning.disabled:hover,.btn-warning[disabled]:hover,fieldset[disabled] .btn-warning:hover,.btn-warning.disabled:focus,.btn-warning[disabled]:focus,fieldset[disabled] .btn-warning:focus,.btn-warning.disabled:active,.btn-warning[disabled]:active,fieldset[disabled] .btn-warning:active,.btn-warning.disabled.active,.btn-warning[disabled].active,fieldset[disabled] .btn-warning.active{background-color:#f0ad4e;border-color:#eea236}.btn-danger{color:#fff;background-color:#d9534f;border-color:#d43f3a}.btn-danger:hover,.btn-danger:focus,.btn-danger:active,.btn-danger.active,.open .dropdown-toggle.btn-danger{col
 or:#fff;background-color:#d2322d;border-color:#ac2925}.btn-danger:active,.btn-danger.active,.open .dropdown-toggle.btn-danger{background-image:none}.btn-danger.disabled,.btn-danger[disabled],fieldset[disabled] .btn-danger,.btn-danger.disabled:hover,.btn-danger[disabled]:hover,fieldset[disabled] .btn-danger:hover,.btn-danger.disabled:focus,.btn-danger[disabled]:focus,fieldset[disabled] .btn-danger:focus,.btn-danger.disabled:active,.btn-danger[disabled]:active,fieldset[disabled] .btn-danger:active,.btn-danger.disabled.active,.btn-danger[disabled].active,fieldset[disabled] .btn-danger.active{background-color:#d9534f;border-color:#d43f3a}.btn-success{color:#fff;background-color:#5cb85c;border-color:#4cae4c}.btn-success:hover,.btn-success:focus,.btn-success:active,.btn-success.active,.open .dropdown-toggle.btn-success{color:#fff;background-color:#47a447;border-color:#398439}.btn-success:active,.btn-success.active,.open .dropdown-toggle.btn-success{background-image:none}.btn-success.disab
 led,.btn-success[disabled],fieldset[disabled] .btn-success,.btn-success.disabled:hover,.btn-success[disabled]:hover,fieldset[disabled] .btn-success:hover,.btn-success.disabled:focus,.btn-success[disabled]:focus,fieldset[disabled] .btn-success:focus,.btn-success.disabled:active,.btn-success[disabled]:active,fieldset[disabled] .btn-success:active,.btn-success.disabled.active,.btn-success[disabled].active,fieldset[disabled] .btn-success.active{background-color:#5cb85c;border-color:#4cae4c}.btn-info{color:#fff;background-color:#5bc0de;border-color:#46b8da}.btn-info:hover,.btn-info:focus,.btn-info:active,.btn-info.active,.open .dropdown-toggle.btn-info{color:#fff;background-color:#39b3d7;border-color:#269abc}.btn-info:active,.btn-info.active,.open .dropdown-toggle.btn-info{background-image:none}.btn-info.disabled,.btn-info[disabled],fieldset[disabled] .btn-info,.btn-info.disabled:hover,.btn-info[disabled]:hover,fieldset[disabled] .btn-info:hover,.btn-info.disabled:focus,.btn-info[disable
 d]:focus,fieldset[disabled] .btn-info:focus,.btn-info.disabled:active,.btn-info[disabled]:active,fieldset[disabled] .btn-info:active,.btn-info.disabled.active,.btn-info[disabled].active,fieldset[disabled] .btn-info.active{background-color:#5bc0de;border-color:#46b8da}.btn-link{font-weight:normal;color:#428bca;cursor:pointer;border-radius:0}.btn-link,.btn-link:active,.btn-link[disabled],fieldset[disabled] .btn-link{background-color:transparent;-webkit-box-shadow:none;box-shadow:none}.btn-link,.btn-link:hover,.btn-link:focus,.btn-link:active{border-color:transparent}.btn-link:hover,.btn-link:focus{color:#2a6496;text-decoration:underline;background-color:transparent}.btn-link[disabled]:hover,fieldset[disabled] .btn-link:hover,.btn-link[disabled]:focus,fieldset[disabled] .btn-link:focus{color:#999;text-decoration:none}.btn-lg{padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}.btn-sm,.btn-xs{padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}.btn-xs{paddi
 ng:1px 5px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0}.btn-block+.btn-block{margin-top:5px}input[type="submit"].btn-block,input[type="reset"].btn-block,input[type="button"].btn-block{width:100%}.fade{opacity:0;-webkit-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{display:none}.collapse.in{display:block}.collapsing{position:relative;height:0;overflow:hidden;-webkit-transition:height .35s ease;transition:height .35s ease}@font-face{font-family:'Glyphicons Halflings';src:url('../fonts/glyphicons-halflings-regular.eot');src:url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'),url('../fonts/glyphicons-halflings-regular.woff') format('woff'),url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'),url('../fonts/glyphicons-halflings-regular.svg#glyphicons-halflingsregular') format('svg')}.glyphicon{position:relative;top:1px;display:inline-block;font-family:'Glyphicons Halflings';-
 webkit-font-smoothing:antialiased;font-style:normal;font-weight:normal;line-height:1}.glyphicon:empty{width:1em}.glyphicon-asterisk:before{content:"\2a"}.glyphicon-plus:before{content:"\2b"}.glyphicon-euro:before{content:"\20ac"}.glyphicon-minus:before{content:"\2212"}.glyphicon-cloud:before{content:"\2601"}.glyphicon-envelope:before{content:"\2709"}.glyphicon-pencil:before{content:"\270f"}.glyphicon-glass:before{content:"\e001"}.glyphicon-music:before{content:"\e002"}.glyphicon-search:before{content:"\e003"}.glyphicon-heart:before{content:"\e005"}.glyphicon-star:before{content:"\e006"}.glyphicon-star-empty:before{content:"\e007"}.glyphicon-user:before{content:"\e008"}.glyphicon-film:before{content:"\e009"}.glyphicon-th-large:before{content:"\e010"}.glyphicon-th:before{content:"\e011"}.glyphicon-th-list:before{content:"\e012"}.glyphicon-ok:before{content:"\e013"}.glyphicon-remove:before{content:"\e014"}.glyphicon-zoom-in:before{content:"\e015"}.glyphicon-zoom-out:before{content:"\e0
 16"}.glyphicon-off:before{content:"\e017"}.glyphicon-signal:before{content:"\e018"}.glyphicon-cog:before{content:"\e019"}.glyphicon-trash:before{content:"\e020"}.glyphicon-home:before{content:"\e021"}.glyphicon-file:before{content:"\e022"}.glyphicon-time:before{content:"\e023"}.glyphicon-road:before{content:"\e024"}.glyphicon-download-alt:before{content:"\e025"}.glyphicon-download:before{content:"\e026"}.glyphicon-upload:before{content:"\e027"}.glyphicon-inbox:before{content:"\e028"}.glyphicon-play-circle:before{content:"\e029"}.glyphicon-repeat:before{content:"\e030"}.glyphicon-refresh:before{content:"\e031"}.glyphicon-list-alt:before{content:"\e032"}.glyphicon-flag:before{content:"\e034"}.glyphicon-headphones:before{content:"\e035"}.glyphicon-volume-off:before{content:"\e036"}.glyphicon-volume-down:before{content:"\e037"}.glyphicon-volume-up:before{content:"\e038"}.glyphicon-qrcode:before{content:"\e039"}.glyphicon-barcode:before{content:"\e040"}.glyphicon-tag:before{content:"\e04
 1"}.glyphicon-tags:before{content:"\e042"}.glyphicon-book:before{content:"\e043"}.glyphicon-print:before{content:"\e045"}.glyphicon-font:before{content:"\e047"}.glyphicon-bold:before{content:"\e048"}.glyphicon-italic:before{content:"\e049"}.glyphicon-text-height:before{content:"\e050"}.glyphicon-text-width:before{content:"\e051"}.glyphicon-align-left:before{content:"\e052"}.glyphicon-align-center:before{content:"\e053"}.glyphicon-align-right:before{content:"\e054"}.glyphicon-align-justify:before{content:"\e055"}.glyphicon-list:before{content:"\e056"}.glyphicon-indent-left:before{content:"\e057"}.glyphicon-indent-right:before{content:"\e058"}.glyphicon-facetime-video:before{content:"\e059"}.glyphicon-picture:before{content:"\e060"}.glyphicon-map-marker:before{content:"\e062"}.glyphicon-adjust:before{content:"\e063"}.glyphicon-tint:before{content:"\e064"}.glyphicon-edit:before{content:"\e065"}.glyphicon-share:before{content:"\e066"}.glyphicon-check:before{content:"\e067"}.glyphicon-mo
 ve:before{content:"\e068"}.glyphicon-step-backward:before{content:"\e069"}.glyphicon-fast-backward:before{content:"\e070"}.glyphicon-backward:before{content:"\e071"}.glyphicon-play:before{content:"\e072"}.glyphicon-pause:before{content:"\e073"}.glyphicon-stop:before{content:"\e074"}.glyphicon-forward:before{content:"\e075"}.glyphicon-fast-forward:before{content:"\e076"}.glyphicon-step-forward:before{content:"\e077"}.glyphicon-eject:before{content:"\e078"}.glyphicon-chevron-left:before{content:"\e079"}.glyphicon-chevron-right:before{content:"\e080"}.glyphicon-plus-sign:before{content:"\e081"}.glyphicon-minus-sign:before{content:"\e082"}.glyphicon-remove-sign:before{content:"\e083"}.glyphicon-ok-sign:before{content:"\e084"}.glyphicon-question-sign:before{content:"\e085"}.glyphicon-info-sign:before{content:"\e086"}.glyphicon-screenshot:before{content:"\e087"}.glyphicon-remove-circle:before{content:"\e088"}.glyphicon-ok-circle:before{content:"\e089"}.glyphicon-ban-circle:before{content:
 "\e090"}.glyphicon-arrow-left:before{content:"\e091"}.glyphicon-arrow-right:before{content:"\e092"}.glyphicon-arrow-up:before{content:"\e093"}.glyphicon-arrow-down:before{content:"\e094"}.glyphicon-share-alt:before{content:"\e095"}.glyphicon-resize-full:before{content:"\e096"}.glyphicon-resize-small:before{content:"\e097"}.glyphicon-exclamation-sign:before{content:"\e101"}.glyphicon-gift:before{content:"\e102"}.glyphicon-leaf:before{content:"\e103"}.glyphicon-eye-open:before{content:"\e105"}.glyphicon-eye-close:before{content:"\e106"}.glyphicon-warning-sign:before{content:"\e107"}.glyphicon-plane:before{content:"\e108"}.glyphicon-random:before{content:"\e110"}.glyphicon-comment:before{content:"\e111"}.glyphicon-magnet:before{content:"\e112"}.glyphicon-chevron-up:before{content:"\e113"}.glyphicon-chevron-down:before{content:"\e114"}.glyphicon-retweet:before{content:"\e115"}.glyphicon-shopping-cart:before{content:"\e116"}.glyphicon-folder-close:before{content:"\e117"}.glyphicon-folder
 -open:before{content:"\e118"}.glyphicon-resize-vertical:before{content:"\e119"}.glyphicon-resize-horizontal:before{content:"\e120"}.glyphicon-hdd:before{content:"\e121"}.glyphicon-bullhorn:before{content:"\e122"}.glyphicon-certificate:before{content:"\e124"}.glyphicon-thumbs-up:before{content:"\e125"}.glyphicon-thumbs-down:before{content:"\e126"}.glyphicon-hand-right:before{content:"\e127"}.glyphicon-hand-left:before{content:"\e128"}.glyphicon-hand-up:before{content:"\e129"}.glyphicon-hand-down:before{content:"\e130"}.glyphicon-circle-arrow-right:before{content:"\e131"}.glyphicon-circle-arrow-left:before{content:"\e132"}.glyphicon-circle-arrow-up:before{content:"\e133"}.glyphicon-circle-arrow-down:before{content:"\e134"}.glyphicon-globe:before{content:"\e135"}.glyphicon-tasks:before{content:"\e137"}.glyphicon-filter:before{content:"\e138"}.glyphicon-fullscreen:before{content:"\e140"}.glyphicon-dashboard:before{content:"\e141"}.glyphicon-heart-empty:before{content:"\e143"}.glyphicon-
 link:before{content:"\e144"}.glyphicon-phone:before{content:"\e145"}.glyphicon-usd:before{content:"\e148"}.glyphicon-gbp:before{content:"\e149"}.glyphicon-sort:before{content:"\e150"}.glyphicon-sort-by-alphabet:before{content:"\e151"}.glyphicon-sort-by-alphabet-alt:before{content:"\e152"}.glyphicon-sort-by-order:before{content:"\e153"}.glyphicon-sort-by-order-alt:before{content:"\e154"}.glyphicon-sort-by-attributes:before{content:"\e155"}.glyphicon-sort-by-attributes-alt:before{content:"\e156"}.glyphicon-unchecked:before{content:"\e157"}.glyphicon-expand:before{content:"\e158"}.glyphicon-collapse-down:before{content:"\e159"}.glyphicon-collapse-up:before{content:"\e160"}.glyphicon-log-in:before{content:"\e161"}.glyphicon-flash:before{content:"\e162"}.glyphicon-log-out:before{content:"\e163"}.glyphicon-new-window:before{content:"\e164"}.glyphicon-record:before{content:"\e165"}.glyphicon-save:before{content:"\e166"}.glyphicon-open:before{content:"\e167"}.glyphicon-saved:before{content:
 "\e168"}.glyphicon-import:before{content:"\e169"}.glyphicon-export:before{content:"\e170"}.glyphicon-send:before{content:"\e171"}.glyphicon-floppy-disk:before{content:"\e172"}.glyphicon-floppy-saved:before{content:"\e173"}.glyphicon-floppy-remove:before{content:"\e174"}.glyphicon-floppy-save:before{content:"\e175"}.glyphicon-floppy-open:before{content:"\e176"}.glyphicon-credit-card:before{content:"\e177"}.glyphicon-transfer:before{content:"\e178"}.glyphicon-cutlery:before{content:"\e179"}.glyphicon-header:before{content:"\e180"}.glyphicon-compressed:before{content:"\e181"}.glyphicon-earphone:before{content:"\e182"}.glyphicon-phone-alt:before{content:"\e183"}.glyphicon-tower:before{content:"\e184"}.glyphicon-stats:before{content:"\e185"}.glyphicon-sd-video:before{content:"\e186"}.glyphicon-hd-video:before{content:"\e187"}.glyphicon-subtitles:before{content:"\e188"}.glyphicon-sound-stereo:before{content:"\e189"}.glyphicon-sound-dolby:before{content:"\e190"}.glyphicon-sound-5-1:before{
 content:"\e191"}.glyphicon-sound-6-1:before{content:"\e192"}.glyphicon-sound-7-1:before{content:"\e193"}.glyphicon-copyright-mark:before{content:"\e194"}.glyphicon-registration-mark:before{content:"\e195"}.glyphicon-cloud-download:before{content:"\e197"}.glyphicon-cloud-upload:before{content:"\e198"}.glyphicon-tree-conifer:before{content:"\e199"}.glyphicon-tree-deciduous:before{content:"\e200"}.glyphicon-briefcase:before{content:"\1f4bc"}.glyphicon-calendar:before{content:"\1f4c5"}.glyphicon-pushpin:before{content:"\1f4cc"}.glyphicon-paperclip:before{content:"\1f4ce"}.glyphicon-camera:before{content:"\1f4f7"}.glyphicon-lock:before{content:"\1f512"}.glyphicon-bell:before{content:"\1f514"}.glyphicon-bookmark:before{content:"\1f516"}.glyphicon-fire:before{content:"\1f525"}.glyphicon-wrench:before{content:"\1f527"}.caret{display:inline-block;width:0;height:0;margin-left:2px;vertical-align:middle;border-top:4px solid #000;border-right:4px solid transparent;border-bottom:0 dotted;border-l
 eft:4px solid transparent;content:""}.dropdown{position:relative}.dropdown-toggle:focus{outline:0}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;font-size:14px;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);border-radius:4px;-webkit-box-shadow:0 6px 12px rgba(0,0,0,0.175);box-shadow:0 6px 12px rgba(0,0,0,0.175);background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{height:1px;margin:9px 0;overflow:hidden;background-color:#e5e5e5}.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:1.428571429;color:#333;white-space:nowrap}.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus{color:#fff;text-decoration:none;background-color:#428bca}.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#fff;text-decoration:none;background-color:#428bca;outl
 ine:0}.dropdown-menu>.disabled>a,.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{color:#999}.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{text-decoration:none;cursor:not-allowed;background-color:transparent;background-image:none;filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.open>.dropdown-menu{display:block}.open>a{outline:0}.dropdown-header{display:block;padding:3px 20px;font-size:12px;line-height:1.428571429;color:#999}.dropdown-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:990}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0 dotted;border-bottom:4px solid #000;content:""}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}@media(min-width:768px){.navbar-right .dropdown-menu{right:0;left:auto}}.btn-default .caret{border-top-color:#333}.btn-primary .caret,.btn-success .caret,.btn-warning .care
 t,.btn-danger .caret,.btn-info .caret{border-top-color:#fff}.dropup .btn-default .caret{border-bottom-color:#333}.dropup .btn-primary .caret,.dropup .btn-success .caret,.dropup .btn-warning .caret,.dropup .btn-danger .caret,.dropup .btn-info .caret{border-bottom-color:#fff}.btn-group,.btn-group-vertical{position:relative;display:inline-block;vertical-align:middle}.btn-group>.btn,.btn-group-vertical>.btn{position:relative;float:left}.btn-group>.btn:hover,.btn-group-vertical>.btn:hover,.btn-group>.btn:focus,.btn-group-vertical>.btn:focus,.btn-group>.btn:active,.btn-group-vertical>.btn:active,.btn-group>.btn.active,.btn-group-vertical>.btn.active{z-index:2}.btn-group>.btn:focus,.btn-group-vertical>.btn:focus{outline:0}.btn-group .btn+.btn,.btn-group .btn+.btn-group,.btn-group .btn-group+.btn,.btn-group .btn-group+.btn-group{margin-left:-1px}.btn-toolbar:before,.btn-toolbar:after{display:table;content:" "}.btn-toolbar:after{clear:both}.btn-toolbar:before,.btn-toolbar:after{display:table
 ;content:" "}.btn-toolbar:after{clear:both}.btn-toolbar .btn-group{float:left}.btn-toolbar>.btn+.btn,.btn-toolbar>.btn-group+.btn,.btn-toolbar>.btn+.btn-group,.btn-toolbar>.btn-group+.btn-group{margin-left:5px}.btn-group>.btn:not(:first-child):not(:last-child):not(.dropdown-toggle){border-radius:0}.btn-group>.btn:first-child{margin-left:0}.btn-group>.btn:first-child:not(:last-child):not(.dropdown-toggle){border-top-right-radius:0;border-bottom-right-radius:0}.btn-group>.btn:last-child:not(:first-child),.btn-group>.dropdown-toggle:not(:first-child){border-bottom-left-radius:0;border-top-left-radius:0}.btn-group>.btn-group{float:left}.btn-group>.btn-group:not(:first-child):not(:last-child)>.btn{border-radius:0}.btn-group>.btn-group:first-child>.btn:last-child,.btn-group>.btn-group:first-child>.dropdown-toggle{border-top-right-radius:0;border-bottom-right-radius:0}.btn-group>.btn-group:last-child>.btn:first-child{border-bottom-left-radius:0;border-top-left-radius:0}.btn-group .dropdown
 -toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group-xs>.btn{padding:5px 10px;padding:1px 5px;font-size:12px;line-height:1.5;border-radius:3px}.btn-group-sm>.btn{padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}.btn-group-lg>.btn{padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}.btn-group>.btn+.dropdown-toggle{padding-right:8px;padding-left:8px}.btn-group>.btn-lg+.dropdown-toggle{padding-right:12px;padding-left:12px}.btn-group.open .dropdown-toggle{-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,0.125);box-shadow:inset 0 3px 5px rgba(0,0,0,0.125)}.btn .caret{margin-left:0}.btn-lg .caret{border-width:5px 5px 0;border-bottom-width:0}.dropup .btn-lg .caret{border-width:0 5px 5px}.btn-group-vertical>.btn,.btn-group-vertical>.btn-group{display:block;float:none;width:100%;max-width:100%}.btn-group-vertical>.btn-group:before,.btn-group-vertical>.btn-group:after{display:table;content:" "}.btn-group-vertical>.btn-group:after{clear:both}.btn-g
 roup-vertical>.btn-group:before,.btn-group-vertical>.btn-group:after{display:table;content:" "}.btn-group-vertical>.btn-group:after{clear:both}.btn-group-vertical>.btn-group>.btn{float:none}.btn-group-vertical>.btn+.btn,.btn-group-vertical>.btn+.btn-group,.btn-group-vertical>.btn-group+.btn,.btn-group-vertical>.btn-group+.btn-group{margin-top:-1px;margin-left:0}.btn-group-vertical>.btn:not(:first-child):not(:last-child){border-radius:0}.btn-group-vertical>.btn:first-child:not(:last-child){border-top-right-radius:4px;border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn:last-child:not(:first-child){border-top-right-radius:0;border-bottom-left-radius:4px;border-top-left-radius:0}.btn-group-vertical>.btn-group:not(:first-child):not(:last-child)>.btn{border-radius:0}.btn-group-vertical>.btn-group:first-child>.btn:last-child,.btn-group-vertical>.btn-group:first-child>.dropdown-toggle{border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>
 .btn-group:last-child>.btn:first-child{border-top-right-radius:0;border-top-left-radius:0}.btn-group-justified{display:table;width:100%;border-collapse:separate;table-layout:fixed}.btn-group-justified .btn{display:table-cell;float:none;width:1%}[data-toggle="buttons"]>.btn>input[type="radio"],[data-toggle="buttons"]>.btn>input[type="checkbox"]{display:none}.input-group{position:relative;display:table;border-collapse:separate}.input-group.col{float:none;padding-right:0;padding-left:0}.input-group .form-control{width:100%;margin-bottom:0}.input-group-lg>.form-control,.input-group-lg>.input-group-addon,.input-group-lg>.input-group-btn>.btn{height:45px;padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}select.input-group-lg>.form-control,select.input-group-lg>.input-group-addon,select.input-group-lg>.input-group-btn>.btn{height:45px;line-height:45px}textarea.input-group-lg>.form-control,textarea.input-group-lg>.input-group-addon,textarea.input-group-lg>.input-group-btn>
 .btn{height:auto}.input-group-sm>.form-control,.input-group-sm>.input-group-addon,.input-group-sm>.input-group-btn>.btn{height:30px;padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}select.input-group-sm>.form-control,select.input-group-sm>.input-group-addon,select.input-group-sm>.input-group-btn>.btn{height:30px;line-height:30px}textarea.input-group-sm>.form-control,textarea.input-group-sm>.input-group-addon,textarea.input-group-sm>.input-group-btn>.btn{height:auto}.input-group-addon,.input-group-btn,.input-group .form-control{display:table-cell}.input-group-addon:not(:first-child):not(:last-child),.input-group-btn:not(:first-child):not(:last-child),.input-group .form-control:not(:first-child):not(:last-child){border-radius:0}.input-group-addon,.input-group-btn{width:1%;white-space:nowrap;vertical-align:middle}.input-group-addon{padding:6px 12px;font-size:14px;font-weight:normal;line-height:1;text-align:center;background-color:#eee;border:1px solid #ccc;border-radiu
 s:4px}.input-group-addon.input-sm{padding:5px 10px;font-size:12px;border-radius:3px}.input-group-addon.input-lg{padding:10px 16px;font-size:18px;border-radius:6px}.input-group-addon input[type="radio"],.input-group-addon input[type="checkbox"]{margin-top:0}.input-group .form-control:first-child,.input-group-addon:first-child,.input-group-btn:first-child>.btn,.input-group-btn:first-child>.dropdown-toggle,.input-group-btn:last-child>.btn:not(:last-child):not(.dropdown-toggle){border-top-right-radius:0;border-bottom-right-radius:0}.input-group-addon:first-child{border-right:0}.input-group .form-control:last-child,.input-group-addon:last-child,.input-group-btn:last-child>.btn,.input-group-btn:last-child>.dropdown-toggle,.input-group-btn:first-child>.btn:not(:first-child){border-bottom-left-radius:0;border-top-left-radius:0}.input-group-addon:last-child{border-left:0}.input-group-btn{position:relative;white-space:nowrap}.input-group-btn>.btn{position:relative}.input-group-btn>.btn+.btn{m
 argin-left:-4px}.input-group-btn>.btn:hover,.input-group-btn>.btn:active{z-index:2}.nav{padding-left:0;margin-bottom:0;list-style:none}.nav:before,.nav:after{display:table;content:" "}.nav:after{clear:both}.nav:before,.nav:after{display:table;content:" "}.nav:after{clear:both}.nav>li{position:relative;display:block}.nav>li>a{position:relative;display:block;padding:10px 15px}.nav>li>a:hover,.nav>li>a:focus{text-decoration:none;background-color:#eee}.nav>li.disabled>a{color:#999}.nav>li.disabled>a:hover,.nav>li.disabled>a:focus{color:#999;text-decoration:none;cursor:not-allowed;background-color:transparent}.nav .open>a,.nav .open>a:hover,.nav .open>a:focus{background-color:#eee;border-color:#428bca}.nav .nav-divider{height:1px;margin:9px 0;overflow:hidden;background-color:#e5e5e5}.nav>li>a>img{max-width:none}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{float:left;margin-bottom:-1px}.nav-tabs>li>a{margin-right:2px;line-height:1.428571429;border:1px solid transparent;border-radiu
 s:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>li.active>a,.nav-tabs>li.active>a:hover,.nav-tabs>li.active>a:focus{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-tabs.nav-justified{width:100%;border-bottom:0}.nav-tabs.nav-justified>li{float:none}.nav-tabs.nav-justified>li>a{margin-bottom:5px;text-align:center}@media(min-width:768px){.nav-tabs.nav-justified>li{display:table-cell;width:1%}.nav-tabs.nav-justified>li>a{margin-bottom:0}}.nav-tabs.nav-justified>li>a{margin-right:0;border-radius:4px}.nav-tabs.nav-justified>.active>a,.nav-tabs.nav-justified>.active>a:hover,.nav-tabs.nav-justified>.active>a:focus{border:1px solid #ddd}@media(min-width:768px){.nav-tabs.nav-justified>li>a{border-bottom:1px solid #ddd;border-radius:4px 4px 0 0}.nav-tabs.nav-justified>.active>a,.nav-tabs.nav-justified>.active>a:hover,.nav-tabs.nav-justified>.active>a:focus{border-bottom-color:#fff}}.nav-pills>li{float:left}.nav
 -pills>li>a{border-radius:5px}.nav-pills>li+li{margin-left:2px}.nav-pills>li.active>a,.nav-pills>li.active>a:hover,.nav-pills>li.active>a:focus{color:#fff;background-color:#428bca}.nav-stacked>li{float:none}.nav-stacked>li+li{margin-top:2px;margin-left:0}.nav-justified{width:100%}.nav-justified>li{float:none}.nav-justified>li>a{margin-bottom:5px;text-align:center}@media(min-width:768px){.nav-justified>li{display:table-cell;width:1%}.nav-justified>li>a{margin-bottom:0}}.nav-tabs-justified{border-bottom:0}.nav-tabs-justified>li>a{margin-right:0;border-radius:4px}.nav-tabs-justified>.active>a,.nav-tabs-justified>.active>a:hover,.nav-tabs-justified>.active>a:focus{border:1px solid #ddd}@media(min-width:768px){.nav-tabs-justified>li>a{border-bottom:1px solid #ddd;border-radius:4px 4px 0 0}.nav-tabs-justified>.active>a,.nav-tabs-justified>.active>a:hover,.nav-tabs-justified>.active>a:focus{border-bottom-color:#fff}}.tabbable:before,.tabbable:after{display:table;content:" "}.tabbable:after
 {clear:both}.tabbable:before,.tabbable:after{display:table;content:" "}.tabbable:after{clear:both}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.nav .caret{border-top-color:#428bca;border-bottom-color:#428bca}.nav a:hover .caret{border-top-color:#2a6496;border-bottom-color:#2a6496}.nav-tabs .dropdown-menu{margin-top:-1px;border-top-right-radius:0;border-top-left-radius:0}.navbar{position:relative;z-index:1000;min-height:50px;margin-bottom:20px;border:1px solid transparent}.navbar:before,.navbar:after{display:table;content:" "}.navbar:after{clear:both}.navbar:before,.navbar:after{display:table;content:" "}.navbar:after{clear:both}@media(min-width:768px){.navbar{border-radius:4px}}.navbar-header:before,.navbar-header:after{display:table;content:" "}.navbar-header:after{clear:both}.navbar-header:before,.navbar-header:after{display:table;content:" "}.navbar-header:after{clear:both}@media(min-width:768px){.navbar-hea
 der{float:left}}.navbar-collapse{max-height:340px;padding-right:15px;padding-left:15px;overflow-x:visible;border-top:1px solid transparent;box-shadow:inset 0 1px 0 rgba(255,255,255,0.1);-webkit-overflow-scrolling:touch}.navbar-collapse:before,.navbar-collapse:after{display:table;content:" "}.navbar-collapse:after{clear:both}.navbar-collapse:before,.navbar-collapse:after{display:table;content:" "}.navbar-collapse:after{clear:both}.navbar-collapse.in{overflow-y:auto}@media(min-width:768px){.navbar-collapse{width:auto;border-top:0;box-shadow:none}.navbar-collapse.collapse{display:block!important;height:auto!important;padding-bottom:0;overflow:visible!important}.navbar-collapse.in{overflow-y:visible}.navbar-collapse .navbar-nav.navbar-left:first-child{margin-left:-15px}.navbar-collapse .navbar-nav.navbar-right:last-child{margin-right:-15px}.navbar-collapse .navbar-text:last-child{margin-right:0}}.container>.navbar-header,.container>.navbar-collapse{margin-right:-15px;margin-left:-15px}@
 media(min-width:768px){.container>.navbar-header,.container>.navbar-collapse{margin-right:0;margin-left:0}}.navbar-static-top{border-width:0 0 1px}@media(min-width:768px){.navbar-static-top{border-radius:0}}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;border-width:0 0 1px}@media(min-width:768px){.navbar-fixed-top,.navbar-fixed-bottom{border-radius:0}}.navbar-fixed-top{top:0;z-index:1030}.navbar-fixed-bottom{bottom:0;margin-bottom:0}.navbar-brand{float:left;padding:15px 15px;font-size:18px;line-height:20px}.navbar-brand:hover,.navbar-brand:focus{text-decoration:none}@media(min-width:768px){.navbar>.container .navbar-brand{margin-left:-15px}}.navbar-toggle{position:relative;float:right;padding:9px 10px;margin-top:8px;margin-right:15px;margin-bottom:8px;background-color:transparent;border:1px solid transparent;border-radius:4px}.navbar-toggle .icon-bar{display:block;width:22px;height:2px;border-radius:1px}.navbar-toggle .icon-bar+.icon-bar{margin-top:4px}@media(
 min-width:768px){.navbar-toggle{display:none}}.navbar-nav{margin:7.5px -15px}.navbar-nav>li>a{padding-top:10px;padding-bottom:10px;line-height:20px}@media(max-width:767px){.navbar-nav .open .dropdown-menu{position:static;float:none;width:auto;margin-top:0;background-color:transparent;border:0;box-shadow:none}.navbar-nav .open .dropdown-menu>li>a,.navbar-nav .open .dropdown-menu .dropdown-header{padding:5px 15px 5px 25px}.navbar-nav .open .dropdown-menu>li>a{line-height:20px}.navbar-nav .open .dropdown-menu>li>a:hover,.navbar-nav .open .dropdown-menu>li>a:focus{background-image:none}}@media(min-width:768px){.navbar-nav{float:left;margin:0}.navbar-nav>li{float:left}.navbar-nav>li>a{padding-top:15px;padding-bottom:15px}}@media(min-width:768px){.navbar-left{float:left!important}.navbar-right{float:right!important}}.navbar-form{padding:10px 15px;margin-top:8px;margin-right:-15px;margin-bottom:8px;margin-left:-15px;border-top:1px solid transparent;border-bottom:1px solid transparent;-webk
 it-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1)}@media(min-width:768px){.navbar-form .form-group{display:inline-block;margin-bottom:0;vertical-align:middle}.navbar-form .form-control{display:inline-block}.navbar-form .radio,.navbar-form .checkbox{display:inline-block;padding-left:0;margin-top:0;margin-bottom:0}.navbar-form .radio input[type="radio"],.navbar-form .checkbox input[type="checkbox"]{float:none;margin-left:0}}@media(max-width:767px){.navbar-form .form-group{margin-bottom:5px}}@media(min-width:768px){.navbar-form{width:auto;padding-top:0;padding-bottom:0;margin-right:0;margin-left:0;border:0;-webkit-box-shadow:none;box-shadow:none}}.navbar-nav>li>.dropdown-menu{margin-top:0;border-top-right-radius:0;border-top-left-radius:0}.navbar-fixed-bottom .navbar-nav>li>.dropdown-menu{border-bottom-right-radius:0;border-bottom-left-radius:0}.navbar-nav.pull-right>li>.dropdown-
 menu,.navbar-nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar-btn{margin-top:8px;margin-bottom:8px}.navbar-text{float:left;margin-top:15px;margin-bottom:15px}@media(min-width:768px){.navbar-text{margin-right:15px;margin-left:15px}}.navbar-default{background-color:#f8f8f8;border-color:#e7e7e7}.navbar-default .navbar-brand{color:#777}.navbar-default .navbar-brand:hover,.navbar-default .navbar-brand:focus{color:#5e5e5e;background-color:transparent}.navbar-default .navbar-text{color:#777}.navbar-default .navbar-nav>li>a{color:#777}.navbar-default .navbar-nav>li>a:hover,.navbar-default .navbar-nav>li>a:focus{color:#333;background-color:transparent}.navbar-default .navbar-nav>.active>a,.navbar-default .navbar-nav>.active>a:hover,.navbar-default .navbar-nav>.active>a:focus{color:#555;background-color:#e7e7e7}.navbar-default .navbar-nav>.disabled>a,.navbar-default .navbar-nav>.disabled>a:hover,.navbar-default .navbar-nav>.disabled>a:focus{color:#ccc;background-color:transparent}.n
 avbar-default .navbar-toggle{border-color:#ddd}.navbar-default .navbar-toggle:hover,.navbar-default .navbar-toggle:focus{background-color:#ddd}.navbar-default .navbar-toggle .icon-bar{background-color:#ccc}.navbar-default .navbar-collapse,.navbar-default .navbar-form{border-color:#e6e6e6}.navbar-default .navbar-nav>.dropdown>a:hover .caret,.navbar-default .navbar-nav>.dropdown>a:focus .caret{border-top-color:#333;border-bottom-color:#333}.navbar-default .navbar-nav>.open>a,.navbar-default .navbar-nav>.open>a:hover,.navbar-default .navbar-nav>.open>a:focus{color:#555;background-color:#e7e7e7}.navbar-default .navbar-nav>.open>a .caret,.navbar-default .navbar-nav>.open>a:hover .caret,.navbar-default .navbar-nav>.open>a:focus .caret{border-top-color:#555;border-bottom-color:#555}.navbar-default .navbar-nav>.dropdown>a .caret{border-top-color:#777;border-bottom-color:#777}@media(max-width:767px){.navbar-default .navbar-nav .open .dropdown-menu>li>a{color:#777}.navbar-default .navbar-nav 
 .open .dropdown-menu>li>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>li>a:focus{color:#333;background-color:transparent}.navbar-default .navbar-nav .open .dropdown-menu>.active>a,.navbar-default .navbar-nav .open .dropdown-menu>.active>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>.active>a:focus{color:#555;background-color:#e7e7e7}.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a,.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:focus{color:#ccc;background-color:transparent}}.navbar-default .navbar-link{color:#777}.navbar-default .navbar-link:hover{color:#333}.navbar-inverse{background-color:#222;border-color:#080808}.navbar-inverse .navbar-brand{color:#999}.navbar-inverse .navbar-brand:hover,.navbar-inverse .navbar-brand:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-text{color:#999}.navbar-inverse .navbar-nav>li>a{color:#999}.navbar-inverse .navba
 r-nav>li>a:hover,.navbar-inverse .navbar-nav>li>a:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-nav>.active>a,.navbar-inverse .navbar-nav>.active>a:hover,.navbar-inverse .navbar-nav>.active>a:focus{color:#fff;background-color:#080808}.navbar-inverse .navbar-nav>.disabled>a,.navbar-inverse .navbar-nav>.disabled>a:hover,.navbar-inverse .navbar-nav>.disabled>a:focus{color:#444;background-color:transparent}.navbar-inverse .navbar-toggle{border-color:#333}.navbar-inverse .navbar-toggle:hover,.navbar-inverse .navbar-toggle:focus{background-color:#333}.navbar-inverse .navbar-toggle .icon-bar{background-color:#fff}.navbar-inverse .navbar-collapse,.navbar-inverse .navbar-form{border-color:#101010}.navbar-inverse .navbar-nav>.open>a,.navbar-inverse .navbar-nav>.open>a:hover,.navbar-inverse .navbar-nav>.open>a:focus{color:#fff;background-color:#080808}.navbar-inverse .navbar-nav>.dropdown>a:hover .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .nav
 bar-nav>.dropdown>a .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .navbar-nav>.open>a .caret,.navbar-inverse .navbar-nav>.open>a:hover .caret,.navbar-inverse .navbar-nav>.open>a:focus .caret{border-top-color:#fff;border-bottom-color:#fff}@media(max-width:767px){.navbar-inverse .navbar-nav .open .dropdown-menu>.dropdown-header{border-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu>li>a{color:#999}.navbar-inverse .navbar-nav .open .dropdown-menu>li>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>li>a:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a,.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:focus{color:#fff;background-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a,.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a:hover,.navbar-inverse .navbar-nav .open .dropdown-me
 nu>.disabled>a:focus{color:#444;background-color:transparent}}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.breadcrumb{padding:8px 15px;margin-bottom:20px;list-style:none;background-color:#f5f5f5;border-radius:4px}.breadcrumb>li{display:inline-block}.breadcrumb>li+li:before{padding:0 5px;color:#ccc;content:"/\00a0"}.breadcrumb>.active{color:#999}.pagination{display:inline-block;padding-left:0;margin:20px 0;border-radius:4px}.pagination>li{display:inline}.pagination>li>a,.pagination>li>span{position:relative;float:left;padding:6px 12px;margin-left:-1px;line-height:1.428571429;text-decoration:none;background-color:#fff;border:1px solid #ddd}.pagination>li:first-child>a,.pagination>li:first-child>span{margin-left:0;border-bottom-left-radius:4px;border-top-left-radius:4px}.pagination>li:last-child>a,.pagination>li:last-child>span{border-top-right-radius:4px;border-bottom-right-radius:4px}.pagination>li>a:hover,.pagination>li>span:hover,.paginatio
 n>li>a:focus,.pagination>li>span:focus{background-color:#eee}.pagination>.active>a,.pagination>.active>span,.pagination>.active>a:hover,.pagination>.active>span:hover,.pagination>.active>a:focus,.pagination>.active>span:focus{z-index:2;color:#fff;cursor:default;background-color:#428bca;border-color:#428bca}.pagination>.disabled>span,.pagination>.disabled>span:hover,.pagination>.disabled>span:focus,.pagination>.disabled>a,.pagination>.disabled>a:hover,.pagination>.disabled>a:focus{color:#999;cursor:not-allowed;background-color:#fff;border-color:#ddd}.pagination-lg>li>a,.pagination-lg>li>span{padding:10px 16px;font-size:18px}.pagination-lg>li:first-child>a,.pagination-lg>li:first-child>span{border-bottom-left-radius:6px;border-top-left-radius:6px}.pagination-lg>li:last-child>a,.pagination-lg>li:last-child>span{border-top-right-radius:6px;border-bottom-right-radius:6px}.pagination-sm>li>a,.pagination-sm>li>span{padding:5px 10px;font-size:12px}.pagination-sm>li:first-child>a,.pagination
 -sm>li:first-child>span{border-bottom-left-radius:3px;border-top-left-radius:3px}.pagination-sm>li:last-child>a,.pagination-sm>li:last-child>span{border-top-right-radius:3px;border-bottom-right-radius:3px}.pager{padding-left:0;margin:20px 0;text-align:center;list-style:none}.pager:before,.pager:after{display:table;content:" "}.pager:after{clear:both}.pager:before,.pager:after{display:table;content:" "}.pager:after{clear:both}.pager li{display:inline}.pager li>a,.pager li>span{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;border-radius:15px}.pager li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#eee}.pager .next>a,.pager .next>span{float:right}.pager .previous>a,.pager .previous>span{float:left}.pager .disabled>a,.pager .disabled>a:hover,.pager .disabled>a:focus,.pager .disabled>span{color:#999;cursor:not-allowed;background-color:#fff}.label{display:inline;padding:.2em .6em .3em;font-size:75%;font-weight:bold;line-height:1;color:
 #fff;text-align:center;white-space:nowrap;vertical-align:baseline;border-radius:.25em}.label[href]:hover,.label[href]:focus{color:#fff;text-decoration:none;cursor:pointer}.label:empty{display:none}.label-default{background-color:#999}.label-default[href]:hover,.label-default[href]:focus{background-color:#808080}.label-primary{background-color:#428bca}.label-primary[href]:hover,.label-primary[href]:focus{background-color:#3071a9}.label-success{background-color:#5cb85c}.label-success[href]:hover,.label-success[href]:focus{background-color:#449d44}.label-info{background-color:#5bc0de}.label-info[href]:hover,.label-info[href]:focus{background-color:#31b0d5}.label-warning{background-color:#f0ad4e}.label-warning[href]:hover,.label-warning[href]:focus{background-color:#ec971f}.label-danger{background-color:#d9534f}.label-danger[href]:hover,.label-danger[href]:focus{background-color:#c9302c}.badge{display:inline-block;min-width:10px;padding:3px 7px;font-size:12px;font-weight:bold;line-heigh
 t:1;color:#fff;text-align:center;white-space:nowrap;vertical-align:baseline;background-color:#999;border-radius:10px}.badge:empty{display:none}a.badge:hover,a.badge:focus{color:#fff;text-decoration:none;cursor:pointer}.btn .badge{position:relative;top:-1px}a.list-group-item.active>.badge,.nav-pills>.active>a>.badge{color:#428bca;background-color:#fff}.nav-pills>li>a>.badge{margin-left:3px}.jumbotron{padding:30px;margin-bottom:30px;font-size:21px;font-weight:200;line-height:2.1428571435;color:inherit;background-color:#eee}.jumbotron h1{line-height:1;color:inherit}.jumbotron p{line-height:1.4}.container .jumbotron{border-radius:6px}@media screen and (min-width:768px){.jumbotron{padding-top:48px;padding-bottom:48px}.container .jumbotron{padding-right:60px;padding-left:60px}.jumbotron h1{font-size:63px}}.thumbnail{display:inline-block;display:block;height:auto;max-width:100%;padding:4px;margin-bottom:20px;line-height:1.428571429;background-color:#fff;border:1px solid #ddd;border-radius:
 4px;-webkit-transition:all .2s ease-in-out;transition:all .2s ease-in-out}.thumbnail>img{display:block;height:auto;max-width:100%}a.thumbnail:hover,a.thumbnail:focus{border-color:#428bca}.thumbnail>img{margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#333}.alert{padding:15px;margin-bottom:20px;border:1px solid transparent;border-radius:4px}.alert h4{margin-top:0;color:inherit}.alert .alert-link{font-weight:bold}.alert>p,.alert>ul{margin-bottom:0}.alert>p+p{margin-top:5px}.alert-dismissable{padding-right:35px}.alert-dismissable .close{position:relative;top:-2px;right:-21px;color:inherit}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-success hr{border-top-color:#c9e2b3}.alert-success .alert-link{color:#356635}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-info hr{border-top-color:#a6e1ec}.alert-info .alert-link{color:#2d6987}.alert-warning{color:#c09853;background-color:#fcf8e3;border-color:#fbe
 ed5}.alert-warning hr{border-top-color:#f8e5be}.alert-warning .alert-link{color:#a47e3c}.alert-danger{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-danger hr{border-top-color:#e6c1c7}.alert-danger .alert-link{color:#953b39}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f5f5f5;border-radius:4px;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress-bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;background-color:#428bca;-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow
 :inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-transition:width .6s ease;transition:width .6s ease}.progress-striped .progress-bar{background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-size:40px 40px}.progress.act
 ive .progress-bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-bar-success{background-color:#5cb85c}.progress-striped .progress-bar-success{background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%
 ,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-bar-info{background-color:#5bc0de}.progress-striped .progress-bar-info{background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,2
 55,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-bar-warning{background-color:#f0ad4e}.progress-striped .progress-bar-warning{background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(
 255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-bar-danger{background-color:#d9534f}.progress-striped .progress-bar-danger{background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.media,.media-body{overflow:
 hidden;zoom:1}.media,.media .media{margin-top:15px}.media:first-child{margin-top:0}.media-object{display:block}.media-heading{margin:0 0 5px}.media>.pull-left{margin-right:10px}.media>.pull-right{margin-left:10px}.media-list{padding-left:0;list-style:none}.list-group{padding-left:0;margin-bottom:20px}.list-group-item{position:relative;display:block;padding:10px 15px;margin-bottom:-1px;background-color:#fff;border:1px solid #ddd}.list-group-item:first-child{border-top-right-radius:4px;border-top-left-radius:4px}.list-group-item:last-child{margin-bottom:0;border-bottom-right-radius:4px;border-bottom-left-radius:4px}.list-group-item>.badge{float:right}.list-group-item>.badge+.badge{margin-right:5px}a.list-group-item{color:#555}a.list-group-item .list-group-item-heading{color:#333}a.list-group-item:hover,a.list-group-item:focus{text-decoration:none;background-color:#f5f5f5}.list-group-item.active,.list-group-item.active:hover,.list-group-item.active:focus{z-index:2;color:#fff;background
 -color:#428bca;border-color:#428bca}.list-group-item.active .list-group-item-heading,.list-group-item.active:hover .list-group-item-heading,.list-group-item.active:focus .list-group-item-heading{color:inherit}.list-group-item.active .list-group-item-text,.list-group-item.active:hover .list-group-item-text,.list-group-item.active:focus .list-group-item-text{color:#e1edf7}.list-group-item-heading{margin-top:0;margin-bottom:5px}.list-group-item-text{margin-bottom:0;line-height:1.3}.panel{margin-bottom:20px;background-color:#fff;border:1px solid transparent;border-radius:4px;-webkit-box-shadow:0 1px 1px rgba(0,0,0,0.05);box-shadow:0 1px 1px rgba(0,0,0,0.05)}.panel-body{padding:15px}.panel-body:before,.panel-body:after{display:table;content:" "}.panel-body:after{clear:both}.panel-body:before,.panel-body:after{display:table;content:" "}.panel-body:after{clear:both}.panel>.list-group{margin-bottom:0}.panel>.list-group .list-group-item{border-width:1px 0}.panel>.list-group .list-group-item:
 first-child{border-top-right-radius:0;border-top-left-radius:0}.panel>.list-group .list-group-item:last-child{border-bottom:0}.panel-heading+.list-group .list-group-item:first-child{border-top-width:0}.panel>.table{margin-bottom:0}.panel>.panel-body+.table{border-top:1px solid #ddd}.panel-heading{padding:10px 15px;border-bottom:1px solid transparent;border-top-right-radius:3px;border-top-left-radius:3px}.panel-title{margin-top:0;margin-bottom:0;font-size:16px}.panel-title>a{color:inherit}.panel-footer{padding:10px 15px;background-color:#f5f5f5;border-top:1px solid #ddd;border-bottom-right-radius:3px;border-bottom-left-radius:3px}.panel-group .panel{margin-bottom:0;overflow:hidden;border-radius:4px}.panel-group .panel+.panel{margin-top:5px}.panel-group .panel-heading{border-bottom:0}.panel-group .panel-heading+.panel-collapse .panel-body{border-top:1px solid #ddd}.panel-group .panel-footer{border-top:0}.panel-group .panel-footer+.panel-collapse .panel-body{border-bottom:1px solid #dd
 d}.panel-default{border-color:#ddd}.panel-default>.panel-heading{color:#333;background-color:#f5f5f5;border-color:#ddd}.panel-default>.panel-heading+.panel-collapse .panel-body{border-top-color:#ddd}.panel-default>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#ddd}.panel-primary{border-color:#428bca}.panel-primary>.panel-heading{color:#fff;background-color:#428bca;border-color:#428bca}.panel-primary>.panel-heading+.panel-collapse .panel-body{border-top-color:#428bca}.panel-primary>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#428bca}.panel-success{border-color:#d6e9c6}.panel-success>.panel-heading{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.panel-success>.panel-heading+.panel-collapse .panel-body{border-top-color:#d6e9c6}.panel-success>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#d6e9c6}.panel-warning{border-color:#fbeed5}.panel-warning>.panel-heading{color:#c09853;background-color:#fcf8e3;border-color:#fbeed5}.pane
 l-warning>.panel-heading+.panel-collapse .panel-body{border-top-color:#fbeed5}.panel-warning>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#fbeed5}.panel-danger{border-color:#eed3d7}.panel-danger>.panel-heading{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.panel-danger>.panel-heading+.panel-collapse .panel-body{border-top-color:#eed3d7}.panel-danger>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#eed3d7}.panel-info{border-color:#bce8f1}.panel-info>.panel-heading{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.panel-info>.panel-heading+.panel-collapse .panel-body{border-top-color:#bce8f1}.panel-info>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#bce8f1}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-co
 lor:rgba(0,0,0,0.15)}.well-lg{padding:24px;border-radius:6px}.well-sm{padding:9px;border-radius:3px}.close{float:right;font-size:21px;font-weight:bold;line-height:1;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover,.close:focus{color:#000;text-decoration:none;cursor:pointer;opacity:.5;filter:alpha(opacity=50)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.modal-open{overflow:hidden}body.modal-open,.modal-open .navbar-fixed-top,.modal-open .navbar-fixed-bottom{margin-right:15px}.modal{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;display:none;overflow:auto;overflow-y:scroll}.modal.fade .modal-dialog{-webkit-transform:translate(0,-25%);-ms-transform:translate(0,-25%);transform:translate(0,-25%);-webkit-transition:-webkit-transform .3s ease-out;-moz-transition:-moz-transform .3s ease-out;-o-transition:-o-transform .3s ease-out;transition:transform .3s ease-out}.modal.in .modal-dialog{-webkit-
 transform:translate(0,0);-ms-transform:translate(0,0);transform:translate(0,0)}.modal-dialog{z-index:1050;width:auto;padding:10px;margin-right:auto;margin-left:auto}.modal-content{position:relative;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.2);border-radius:6px;outline:0;-webkit-box-shadow:0 3px 9px rgba(0,0,0,0.5);box-shadow:0 3px 9px rgba(0,0,0,0.5);background-clip:padding-box}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1030;background-color:#000}.modal-backdrop.fade{opacity:0;filter:alpha(opacity=0)}.modal-backdrop.in{opacity:.5;filter:alpha(opacity=50)}.modal-header{min-height:16.428571429px;padding:15px;border-bottom:1px solid #e5e5e5}.modal-header .close{margin-top:-2px}.modal-title{margin:0;line-height:1.428571429}.modal-body{position:relative;padding:20px}.modal-footer{padding:19px 20px 20px;margin-top:15px;text-align:right;border-top:1px solid #e5e5e5}.modal-footer:before,.modal-footer:after{display:table;content:" "}.
 modal-footer:after{clear:both}.modal-footer:before,.modal-footer:after{display:table;content:" "}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.modal-footer .btn-block+.btn-block{margin-left:0}@media screen and (min-width:768px){.modal-dialog{right:auto;left:50%;width:600px;padding-top:30px;padding-bottom:30px}.modal-content{-webkit-box-shadow:0 5px 15px rgba(0,0,0,0.5);box-shadow:0 5px 15px rgba(0,0,0,0.5)}}.tooltip{position:absolute;z-index:1030;display:block;font-size:12px;line-height:1.4;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.9;filter:alpha(opacity=90)}.tooltip.top{padding:5px 0;margin-top:-3px}.tooltip.right{padding:0 5px;margin-left:3px}.tooltip.bottom{padding:5px 0;margin-top:3px}.tooltip.left{padding:0 5px;margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;border-ra
 dius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.top-left .tooltip-arrow{bottom:0;left:5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.top-right .tooltip-arrow{right:5px;bottom:0;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.tooltip.bottom-left .tooltip-arrow{top:0;left:5px;border-bottom-color:#000;border-width:0 5px 5px}.tooltip.bottom-right .tooltip-arrow{top:0;right:5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;displ
 ay:none;max-width:276px;padding:1px;text-align:left;white-space:normal;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);background-clip:padding-box}.popover.top{margin-top:-10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-left:-10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover .arrow,.popover .arrow:after{position:absolute;display:block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow{border-width:11px}.popover .arrow:after{border-width:10px;content:""}.popover.top .arrow{bottom:-11px;left:50%;margin-left:-11px;border-top-color:#999;border-top-color:rgba(0,0,0,0.25);border-bottom-width:0}.popover.top .arrow:after{bott
 om:1px;margin-left:-10px;border-top-color:#fff;border-bottom-width:0;content:" "}.popover.right .arrow{top:50%;left:-11px;margin-top:-11px;border-right-color:#999;border-right-color:rgba(0,0,0,0.25);border-left-width:0}.popover.right .arrow:after{bottom:-10px;left:1px;border-right-color:#fff;border-left-width:0;content:" "}.popover.bottom .arrow{top:-11px;left:50%;margin-left:-11px;border-bottom-color:#999;border-bottom-color:rgba(0,0,0,0.25);border-top-width:0}.popover.bottom .arrow:after{top:1px;margin-left:-10px;border-bottom-color:#fff;border-top-width:0;content:" "}.popover.left .arrow{top:50%;right:-11px;margin-top:-11px;border-left-color:#999;border-left-color:rgba(0,0,0,0.25);border-right-width:0}.popover.left .arrow:after{right:1px;bottom:-10px;border-left-color:#fff;border-right-width:0;content:" "}.carousel{position:relative}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel-inner>.item{position:relative;display:none;-webkit-transition:.6s ease-in-out 
 left;transition:.6s ease-in-out left}.carousel-inner>.item>img,.carousel-inner>.item>a>img{display:block;height:auto;max-width:100%;line-height:1}.carousel-inner>.active,.carousel-inner>.next,.carousel-inner>.prev{display:block}.carousel-inner>.active{left:0}.carousel-inner>.next,.carousel-inner>.prev{position:absolute;top:0;width:100%}.carousel-inner>.next{left:100%}.carousel-inner>.prev{left:-100%}.carousel-inner>.next.left,.carousel-inner>.prev.right{left:0}.carousel-inner>.active.left{left:-100%}.carousel-inner>.active.right{left:100%}.carousel-control{position:absolute;top:0;bottom:0;left:0;width:15%;font-size:20px;color:#fff;text-align:center;text-shadow:0 1px 2px rgba(0,0,0,0.6);opacity:.5;filter:alpha(opacity=50)}.carousel-control.left{background-image:-webkit-gradient(linear,0 top,100% top,from(rgba(0,0,0,0.5)),to(rgba(0,0,0,0.0001)));background-image:-webkit-linear-gradient(left,color-stop(rgba(0,0,0,0.5) 0),color-stop(rgba(0,0,0,0.0001) 100%));background-image:-moz-linear
 -gradient(left,rgba(0,0,0,0.5) 0,rgba(0,0,0,0.0001) 100%);background-image:linear-gradient(to right,rgba(0,0,0,0.5) 0,rgba(0,0,0,0.0001) 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#80000000',endColorstr='#00000000',GradientType=1)}.carousel-control.right{right:0;left:auto;background-image:-webkit-gradient(linear,0 top,100% top,from(rgba(0,0,0,0.0001)),to(rgba(0,0,0,0.5)));background-image:-webkit-linear-gradient(left,color-stop(rgba(0,0,0,0.0001) 0),color-stop(rgba(0,0,0,0.5) 100%));background-image:-moz-linear-gradient(left,rgba(0,0,0,0.0001) 0,rgba(0,0,0,0.5) 100%);background-image:linear-gradient(to right,rgba(0,0,0,0.0001) 0,rgba(0,0,0,0.5) 100%);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000',endColorstr='#80000000',GradientType=1)}.carousel-control:hover,.carousel-control:focus{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-control .icon-
 prev,.carousel-control .icon-next,.carousel-control .glyphicon-chevron-left,.carousel-contr

<TRUNCATED>


[22/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/css/font-awesome.min.css
----------------------------------------------------------------------
diff --git a/gemfire-site/content/css/font-awesome.min.css b/gemfire-site/content/css/font-awesome.min.css
deleted file mode 100644
index 7b8ed50..0000000
--- a/gemfire-site/content/css/font-awesome.min.css
+++ /dev/null
@@ -1,405 +0,0 @@
-@font-face{font-family:'FontAwesome';src:url('../font/fontawesome-webfont-eot.eot?v=3.2.1');src:url('../font/fontawesome-webfont-eot.eot?#iefix&v=3.2.1') format('embedded-opentype'),url('../font/fontawesome-webfont-woff.woff?v=3.2.1') format('woff'),url('../font/fontawesome-webfont-ttf.ttf?v=3.2.1') format('truetype'),url('../font/fontawesome-webfont-svg.svg#fontawesomeregular?v=3.2.1') format('svg');font-weight:normal;font-style:normal;}[class^="icon-"],[class*=" icon-"]{font-family:FontAwesome;font-weight:normal;font-style:normal;text-decoration:inherit;-webkit-font-smoothing:antialiased;*margin-right:.3em;}
-[class^="icon-"]:before,[class*=" icon-"]:before{text-decoration:inherit;display:inline-block;speak:none;}
-.icon-large:before{vertical-align:-10%;font-size:1.3333333333333333em;}
-a [class^="icon-"],a [class*=" icon-"]{display:inline;}
-[class^="icon-"].icon-fixed-width,[class*=" icon-"].icon-fixed-width{display:inline-block;width:1.1428571428571428em;text-align:right;padding-right:0.2857142857142857em;}[class^="icon-"].icon-fixed-width.icon-large,[class*=" icon-"].icon-fixed-width.icon-large{width:1.4285714285714286em;}
-.icons-ul{margin-left:2.142857142857143em;list-style-type:none;}.icons-ul>li{position:relative;}
-.icons-ul .icon-li{position:absolute;left:-2.142857142857143em;width:2.142857142857143em;text-align:center;line-height:inherit;}
-[class^="icon-"].hide,[class*=" icon-"].hide{display:none;}
-.icon-muted{color:#eeeeee;}
-.icon-light{color:#ffffff;}
-.icon-dark{color:#333333;}
-.icon-border{border:solid 1px #eeeeee;padding:.2em .25em .15em;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;}
-.icon-2x{font-size:2em;}.icon-2x.icon-border{border-width:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}
-.icon-3x{font-size:3em;}.icon-3x.icon-border{border-width:3px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px;}
-.icon-4x{font-size:4em;}.icon-4x.icon-border{border-width:4px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;}
-.icon-5x{font-size:5em;}.icon-5x.icon-border{border-width:5px;-webkit-border-radius:7px;-moz-border-radius:7px;border-radius:7px;}
-.pull-right{float:right;}
-.pull-left{float:left;}
-[class^="icon-"].pull-left,[class*=" icon-"].pull-left{margin-right:.3em;}
-[class^="icon-"].pull-right,[class*=" icon-"].pull-right{margin-left:.3em;}
-[class^="icon-"],[class*=" icon-"]{display:inline;width:auto;height:auto;line-height:normal;vertical-align:baseline;background-image:none;background-position:0% 0%;background-repeat:repeat;margin-top:0;}
-.icon-white,.nav-pills>.active>a>[class^="icon-"],.nav-pills>.active>a>[class*=" icon-"],.nav-list>.active>a>[class^="icon-"],.nav-list>.active>a>[class*=" icon-"],.navbar-inverse .nav>.active>a>[class^="icon-"],.navbar-inverse .nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"],.dropdown-submenu:hover>a>[class^="icon-"],.dropdown-submenu:hover>a>[class*=" icon-"]{background-image:none;}
-.btn [class^="icon-"].icon-large,.nav [class^="icon-"].icon-large,.btn [class*=" icon-"].icon-large,.nav [class*=" icon-"].icon-large{line-height:.9em;}
-.btn [class^="icon-"].icon-spin,.nav [class^="icon-"].icon-spin,.btn [class*=" icon-"].icon-spin,.nav [class*=" icon-"].icon-spin{display:inline-block;}
-.nav-tabs [class^="icon-"],.nav-pills [class^="icon-"],.nav-tabs [class*=" icon-"],.nav-pills [class*=" icon-"],.nav-tabs [class^="icon-"].icon-large,.nav-pills [class^="icon-"].icon-large,.nav-tabs [class*=" icon-"].icon-large,.nav-pills [class*=" icon-"].icon-large{line-height:.9em;}
-.btn [class^="icon-"].pull-left.icon-2x,.btn [class*=" icon-"].pull-left.icon-2x,.btn [class^="icon-"].pull-right.icon-2x,.btn [class*=" icon-"].pull-right.icon-2x{margin-top:.18em;}
-.btn [class^="icon-"].icon-spin.icon-large,.btn [class*=" icon-"].icon-spin.icon-large{line-height:.8em;}
-.btn.btn-small [class^="icon-"].pull-left.icon-2x,.btn.btn-small [class*=" icon-"].pull-left.icon-2x,.btn.btn-small [class^="icon-"].pull-right.icon-2x,.btn.btn-small [class*=" icon-"].pull-right.icon-2x{margin-top:.25em;}
-.btn.btn-large [class^="icon-"],.btn.btn-large [class*=" icon-"]{margin-top:0;}.btn.btn-large [class^="icon-"].pull-left.icon-2x,.btn.btn-large [class*=" icon-"].pull-left.icon-2x,.btn.btn-large [class^="icon-"].pull-right.icon-2x,.btn.btn-large [class*=" icon-"].pull-right.icon-2x{margin-top:.05em;}
-.btn.btn-large [class^="icon-"].pull-left.icon-2x,.btn.btn-large [class*=" icon-"].pull-left.icon-2x{margin-right:.2em;}
-.btn.btn-large [class^="icon-"].pull-right.icon-2x,.btn.btn-large [class*=" icon-"].pull-right.icon-2x{margin-left:.2em;}
-.nav-list [class^="icon-"],.nav-list [class*=" icon-"]{line-height:inherit;}
-.icon-stack{position:relative;display:inline-block;width:2em;height:2em;line-height:2em;vertical-align:-35%;}.icon-stack [class^="icon-"],.icon-stack [class*=" icon-"]{display:block;text-align:center;position:absolute;width:100%;height:100%;font-size:1em;line-height:inherit;*line-height:2em;}
-.icon-stack .icon-stack-base{font-size:2em;*line-height:1em;}
-.icon-spin{display:inline-block;-moz-animation:spin 2s infinite linear;-o-animation:spin 2s infinite linear;-webkit-animation:spin 2s infinite linear;animation:spin 2s infinite linear;}
-a .icon-stack,a .icon-spin{display:inline-block;text-decoration:none;}
-@-moz-keyframes spin{0%{-moz-transform:rotate(0deg);} 100%{-moz-transform:rotate(359deg);}}@-webkit-keyframes spin{0%{-webkit-transform:rotate(0deg);} 100%{-webkit-transform:rotate(359deg);}}@-o-keyframes spin{0%{-o-transform:rotate(0deg);} 100%{-o-transform:rotate(359deg);}}@-ms-keyframes spin{0%{-ms-transform:rotate(0deg);} 100%{-ms-transform:rotate(359deg);}}@keyframes spin{0%{transform:rotate(0deg);} 100%{transform:rotate(359deg);}}.icon-rotate-90:before{-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg);filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=1);}
-.icon-rotate-180:before{-webkit-transform:rotate(180deg);-moz-transform:rotate(180deg);-ms-transform:rotate(180deg);-o-transform:rotate(180deg);transform:rotate(180deg);filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=2);}
-.icon-rotate-270:before{-webkit-transform:rotate(270deg);-moz-transform:rotate(270deg);-ms-transform:rotate(270deg);-o-transform:rotate(270deg);transform:rotate(270deg);filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=3);}
-.icon-flip-horizontal:before{-webkit-transform:scale(-1, 1);-moz-transform:scale(-1, 1);-ms-transform:scale(-1, 1);-o-transform:scale(-1, 1);transform:scale(-1, 1);}
-.icon-flip-vertical:before{-webkit-transform:scale(1, -1);-moz-transform:scale(1, -1);-ms-transform:scale(1, -1);-o-transform:scale(1, -1);transform:scale(1, -1);}
-a .icon-rotate-90:before,a .icon-rotate-180:before,a .icon-rotate-270:before,a .icon-flip-horizontal:before,a .icon-flip-vertical:before{display:inline-block;}
-.icon-glass:before{content:"\f000";}
-.icon-music:before{content:"\f001";}
-.icon-search:before{content:"\f002";}
-.icon-envelope-alt:before{content:"\f003";}
-.icon-heart:before{content:"\f004";}
-.icon-star:before{content:"\f005";}
-.icon-star-empty:before{content:"\f006";}
-.icon-user:before{content:"\f007";}
-.icon-film:before{content:"\f008";}
-.icon-th-large:before{content:"\f009";}
-.icon-th:before{content:"\f00a";}
-.icon-th-list:before{content:"\f00b";}
-.icon-ok:before{content:"\f00c";}
-.icon-remove:before{content:"\f00d";}
-.icon-zoom-in:before{content:"\f00e";}
-.icon-zoom-out:before{content:"\f010";}
-.icon-power-off:before,.icon-off:before{content:"\f011";}
-.icon-signal:before{content:"\f012";}
-.icon-gear:before,.icon-cog:before{content:"\f013";}
-.icon-trash:before{content:"\f014";}
-.icon-home:before{content:"\f015";}
-.icon-file-alt:before{content:"\f016";}
-.icon-time:before{content:"\f017";}
-.icon-road:before{content:"\f018";}
-.icon-download-alt:before{content:"\f019";}
-.icon-download:before{content:"\f01a";}
-.icon-upload:before{content:"\f01b";}
-.icon-inbox:before{content:"\f01c";}
-.icon-play-circle:before{content:"\f01d";}
-.icon-rotate-right:before,.icon-repeat:before{content:"\f01e";}
-.icon-refresh:before{content:"\f021";}
-.icon-list-alt:before{content:"\f022";}
-.icon-lock:before{content:"\f023";}
-.icon-flag:before{content:"\f024";}
-.icon-headphones:before{content:"\f025";}
-.icon-volume-off:before{content:"\f026";}
-.icon-volume-down:before{content:"\f027";}
-.icon-volume-up:before{content:"\f028";}
-.icon-qrcode:before{content:"\f029";}
-.icon-barcode:before{content:"\f02a";}
-.icon-tag:before{content:"\f02b";}
-.icon-tags:before{content:"\f02c";}
-.icon-book:before{content:"\f02d";}
-.icon-bookmark:before{content:"\f02e";}
-.icon-print:before{content:"\f02f";}
-.icon-camera:before{content:"\f030";}
-.icon-font:before{content:"\f031";}
-.icon-bold:before{content:"\f032";}
-.icon-italic:before{content:"\f033";}
-.icon-text-height:before{content:"\f034";}
-.icon-text-width:before{content:"\f035";}
-.icon-align-left:before{content:"\f036";}
-.icon-align-center:before{content:"\f037";}
-.icon-align-right:before{content:"\f038";}
-.icon-align-justify:before{content:"\f039";}
-.icon-list:before{content:"\f03a";}
-.icon-indent-left:before{content:"\f03b";}
-.icon-indent-right:before{content:"\f03c";}
-.icon-facetime-video:before{content:"\f03d";}
-.icon-picture:before{content:"\f03e";}
-.icon-pencil:before{content:"\f040";}
-.icon-map-marker:before{content:"\f041";}
-.icon-adjust:before{content:"\f042";}
-.icon-tint:before{content:"\f043";}
-.icon-edit:before{content:"\f044";}
-.icon-share:before{content:"\f045";}
-.icon-check:before{content:"\f046";}
-.icon-move:before{content:"\f047";}
-.icon-step-backward:before{content:"\f048";}
-.icon-fast-backward:before{content:"\f049";}
-.icon-backward:before{content:"\f04a";}
-.icon-play:before{content:"\f04b";}
-.icon-pause:before{content:"\f04c";}
-.icon-stop:before{content:"\f04d";}
-.icon-forward:before{content:"\f04e";}
-.icon-fast-forward:before{content:"\f050";}
-.icon-step-forward:before{content:"\f051";}
-.icon-eject:before{content:"\f052";}
-.icon-chevron-left:before{content:"\f053";}
-.icon-chevron-right:before{content:"\f054";}
-.icon-plus-sign:before{content:"\f055";}
-.icon-minus-sign:before{content:"\f056";}
-.icon-remove-sign:before{content:"\f057";}
-.icon-ok-sign:before{content:"\f058";}
-.icon-question-sign:before{content:"\f059";}
-.icon-info-sign:before{content:"\f05a";}
-.icon-screenshot:before{content:"\f05b";}
-.icon-remove-circle:before{content:"\f05c";}
-.icon-ok-circle:before{content:"\f05d";}
-.icon-ban-circle:before{content:"\f05e";}
-.icon-arrow-left:before{content:"\f060";}
-.icon-arrow-right:before{content:"\f061";}
-.icon-arrow-up:before{content:"\f062";}
-.icon-arrow-down:before{content:"\f063";}
-.icon-mail-forward:before,.icon-share-alt:before{content:"\f064";}
-.icon-resize-full:before{content:"\f065";}
-.icon-resize-small:before{content:"\f066";}
-.icon-plus:before{content:"\f067";}
-.icon-minus:before{content:"\f068";}
-.icon-asterisk:before{content:"\f069";}
-.icon-exclamation-sign:before{content:"\f06a";}
-.icon-gift:before{content:"\f06b";}
-.icon-leaf:before{content:"\f06c";}
-.icon-fire:before{content:"\f06d";}
-.icon-eye-open:before{content:"\f06e";}
-.icon-eye-close:before{content:"\f070";}
-.icon-warning-sign:before{content:"\f071";}
-.icon-plane:before{content:"\f072";}
-.icon-calendar:before{content:"\f073";}
-.icon-random:before{content:"\f074";}
-.icon-comment:before{content:"\f075";}
-.icon-magnet:before{content:"\f076";}
-.icon-chevron-up:before{content:"\f077";}
-.icon-chevron-down:before{content:"\f078";}
-.icon-retweet:before{content:"\f079";}
-.icon-shopping-cart:before{content:"\f07a";}
-.icon-folder-close:before{content:"\f07b";}
-.icon-folder-open:before{content:"\f07c";}
-.icon-resize-vertical:before{content:"\f07d";}
-.icon-resize-horizontal:before{content:"\f07e";}
-.icon-bar-chart:before{content:"\f080";}
-.icon-twitter-sign:before{content:"\f081";}
-.icon-facebook-sign:before{content:"\f082";}
-.icon-camera-retro:before{content:"\f083";}
-.icon-key:before{content:"\f084";}
-.icon-gears:before,.icon-cogs:before{content:"\f085";}
-.icon-comments:before{content:"\f086";}
-.icon-thumbs-up-alt:before{content:"\f087";}
-.icon-thumbs-down-alt:before{content:"\f088";}
-.icon-star-half:before{content:"\f089";}
-.icon-heart-empty:before{content:"\f08a";}
-.icon-signout:before{content:"\f08b";}
-.icon-linkedin-sign:before{content:"\f08c";}
-.icon-pushpin:before{content:"\f08d";}
-.icon-external-link:before{content:"\f08e";}
-.icon-signin:before{content:"\f090";}
-.icon-trophy:before{content:"\f091";}
-.icon-github-sign:before{content:"\f092";}
-.icon-upload-alt:before{content:"\f093";}
-.icon-lemon:before{content:"\f094";}
-.icon-phone:before{content:"\f095";}
-.icon-unchecked:before,.icon-check-empty:before{content:"\f096";}
-.icon-bookmark-empty:before{content:"\f097";}
-.icon-phone-sign:before{content:"\f098";}
-.icon-twitter:before{content:"\f099";}
-.icon-facebook:before{content:"\f09a";}
-.icon-github:before{content:"\f09b";}
-.icon-wiki:before{content:"\f109";}
-.icon-releases:before{content:"\f01a";}
-.icon-unlock:before{content:"\f09c";}
-.icon-credit-card:before{content:"\f09d";}
-.icon-rss:before{content:"\f09e";}
-.icon-hdd:before{content:"\f0a0";}
-.icon-bullhorn:before{content:"\f0a1";}
-.icon-bell:before{content:"\f0a2";}
-.icon-certificate:before{content:"\f0a3";}
-.icon-hand-right:before{content:"\f0a4";}
-.icon-hand-left:before{content:"\f0a5";}
-.icon-hand-up:before{content:"\f0a6";}
-.icon-hand-down:before{content:"\f0a7";}
-.icon-circle-arrow-left:before{content:"\f0a8";}
-.icon-circle-arrow-right:before{content:"\f0a9";}
-.icon-circle-arrow-up:before{content:"\f0aa";}
-.icon-circle-arrow-down:before{content:"\f0ab";}
-.icon-globe:before{content:"\f0ac";}
-.icon-wrench:before{content:"\f0ad";}
-.icon-tasks:before{content:"\f0ae";}
-.icon-filter:before{content:"\f0b0";}
-.icon-briefcase:before{content:"\f0b1";}
-.icon-fullscreen:before{content:"\f0b2";}
-.icon-group:before{content:"\f0c0";}
-.icon-link:before{content:"\f0c1";}
-.icon-cloud:before{content:"\f0c2";}
-.icon-beaker:before{content:"\f0c3";}
-.icon-cut:before{content:"\f0c4";}
-.icon-copy:before{content:"\f0c5";}
-.icon-paperclip:before,.icon-paper-clip:before{content:"\f0c6";}
-.icon-save:before{content:"\f0c7";}
-.icon-sign-blank:before{content:"\f0c8";}
-.icon-reorder:before{content:"\f0c9";}
-.icon-list-ul:before{content:"\f0ca";}
-.icon-list-ol:before{content:"\f0cb";}
-.icon-strikethrough:before{content:"\f0cc";}
-.icon-underline:before{content:"\f0cd";}
-.icon-table:before{content:"\f0ce";}
-.icon-magic:before{content:"\f0d0";}
-.icon-truck:before{content:"\f0d1";}
-.icon-pinterest:before{content:"\f0d2";}
-.icon-pinterest-sign:before{content:"\f0d3";}
-.icon-google-plus-sign:before{content:"\f0d4";}
-.icon-google-plus:before{content:"\f0d5";}
-.icon-money:before{content:"\f0d6";}
-.icon-caret-down:before{content:"\f0d7";}
-.icon-caret-up:before{content:"\f0d8";}
-.icon-caret-left:before{content:"\f0d9";}
-.icon-caret-right:before{content:"\f0da";}
-.icon-columns:before{content:"\f0db";}
-.icon-sort:before{content:"\f0dc";}
-.icon-sort-down:before{content:"\f0dd";}
-.icon-sort-up:before{content:"\f0de";}
-.icon-envelope:before{content:"\f0e0";}
-.icon-linkedin:before{content:"\f0e1";}
-.icon-rotate-left:before,.icon-undo:before{content:"\f0e2";}
-.icon-legal:before{content:"\f0e3";}
-.icon-dashboard:before{content:"\f0e4";}
-.icon-comment-alt:before{content:"\f0e5";}
-.icon-comments-alt:before{content:"\f0e6";}
-.icon-bolt:before{content:"\f0e7";}
-.icon-sitemap:before{content:"\f0e8";}
-.icon-umbrella:before{content:"\f0e9";}
-.icon-paste:before{content:"\f0ea";}
-.icon-lightbulb:before{content:"\f0eb";}
-.icon-exchange:before{content:"\f0ec";}
-.icon-cloud-download:before{content:"\f0ed";}
-.icon-cloud-upload:before{content:"\f0ee";}
-.icon-user-md:before{content:"\f0f0";}
-.icon-stethoscope:before{content:"\f0f1";}
-.icon-suitcase:before{content:"\f0f2";}
-.icon-bell-alt:before{content:"\f0f3";}
-.icon-coffee:before{content:"\f0f4";}
-.icon-food:before{content:"\f0f5";}
-.icon-file-text-alt:before{content:"\f0f6";}
-.icon-building:before{content:"\f0f7";}
-.icon-hospital:before{content:"\f0f8";}
-.icon-ambulance:before{content:"\f0f9";}
-.icon-medkit:before{content:"\f0fa";}
-.icon-fighter-jet:before{content:"\f0fb";}
-.icon-beer:before{content:"\f0fc";}
-.icon-h-sign:before{content:"\f0fd";}
-.icon-plus-sign-alt:before{content:"\f0fe";}
-.icon-double-angle-left:before{content:"\f100";}
-.icon-double-angle-right:before{content:"\f101";}
-.icon-double-angle-up:before{content:"\f102";}
-.icon-double-angle-down:before{content:"\f103";}
-.icon-angle-left:before{content:"\f104";}
-.icon-angle-right:before{content:"\f105";}
-.icon-angle-up:before{content:"\f106";}
-.icon-angle-down:before{content:"\f107";}
-.icon-desktop:before{content:"\f108";}
-.icon-laptop:before{content:"\f109";}
-.icon-tablet:before{content:"\f10a";}
-.icon-mobile-phone:before{content:"\f10b";}
-.icon-circle-blank:before{content:"\f10c";}
-.icon-quote-left:before{content:"\f10d";}
-.icon-quote-right:before{content:"\f10e";}
-.icon-spinner:before{content:"\f110";}
-.icon-circle:before{content:"\f111";}
-.icon-mail-reply:before,.icon-reply:before{content:"\f112";}
-.icon-github-alt:before{content:"\f113";}
-.icon-folder-close-alt:before{content:"\f114";}
-.icon-folder-open-alt:before{content:"\f115";}
-.icon-expand-alt:before{content:"\f116";}
-.icon-collapse-alt:before{content:"\f117";}
-.icon-smile:before{content:"\f118";}
-.icon-frown:before{content:"\f119";}
-.icon-meh:before{content:"\f11a";}
-.icon-gamepad:before{content:"\f11b";}
-.icon-keyboard:before{content:"\f11c";}
-.icon-flag-alt:before{content:"\f11d";}
-.icon-flag-checkered:before{content:"\f11e";}
-.icon-terminal:before{content:"\f120";}
-.icon-code:before{content:"\f121";}
-.icon-reply-all:before{content:"\f122";}
-.icon-mail-reply-all:before{content:"\f122";}
-.icon-star-half-full:before,.icon-star-half-empty:before{content:"\f123";}
-.icon-location-arrow:before{content:"\f124";}
-.icon-crop:before{content:"\f125";}
-.icon-code-fork:before{content:"\f126";}
-.icon-unlink:before{content:"\f127";}
-.icon-question:before{content:"\f128";}
-.icon-info:before{content:"\f129";}
-.icon-exclamation:before{content:"\f12a";}
-.icon-superscript:before{content:"\f12b";}
-.icon-subscript:before{content:"\f12c";}
-.icon-eraser:before{content:"\f12d";}
-.icon-puzzle-piece:before{content:"\f12e";}
-.icon-microphone:before{content:"\f130";}
-.icon-microphone-off:before{content:"\f131";}
-.icon-shield:before{content:"\f132";}
-.icon-calendar-empty:before{content:"\f133";}
-.icon-fire-extinguisher:before{content:"\f134";}
-.icon-rocket:before{content:"\f135";}
-.icon-maxcdn:before{content:"\f136";}
-.icon-chevron-sign-left:before{content:"\f137";}
-.icon-chevron-sign-right:before{content:"\f138";}
-.icon-chevron-sign-up:before{content:"\f139";}
-.icon-chevron-sign-down:before{content:"\f13a";}
-.icon-html5:before{content:"\f13b";}
-.icon-css3:before{content:"\f13c";}
-.icon-anchor:before{content:"\f13d";}
-.icon-unlock-alt:before{content:"\f13e";}
-.icon-bullseye:before{content:"\f140";}
-.icon-ellipsis-horizontal:before{content:"\f141";}
-.icon-ellipsis-vertical:before{content:"\f142";}
-.icon-rss-sign:before{content:"\f143";}
-.icon-play-sign:before{content:"\f144";}
-.icon-ticket:before{content:"\f145";}
-.icon-minus-sign-alt:before{content:"\f146";}
-.icon-check-minus:before{content:"\f147";}
-.icon-level-up:before{content:"\f148";}
-.icon-level-down:before{content:"\f149";}
-.icon-check-sign:before{content:"\f14a";}
-.icon-edit-sign:before{content:"\f14b";}
-.icon-external-link-sign:before{content:"\f14c";}
-.icon-share-sign:before{content:"\f14d";}
-.icon-compass:before{content:"\f14e";}
-.icon-collapse:before{content:"\f150";}
-.icon-collapse-top:before{content:"\f151";}
-.icon-expand:before{content:"\f152";}
-.icon-euro:before,.icon-eur:before{content:"\f153";}
-.icon-gbp:before{content:"\f154";}
-.icon-dollar:before,.icon-usd:before{content:"\f155";}
-.icon-rupee:before,.icon-inr:before{content:"\f156";}
-.icon-yen:before,.icon-jpy:before{content:"\f157";}
-.icon-renminbi:before,.icon-cny:before{content:"\f158";}
-.icon-won:before,.icon-krw:before{content:"\f159";}
-.icon-bitcoin:before,.icon-btc:before{content:"\f15a";}
-.icon-file:before{content:"\f15b";}
-.icon-file-text:before{content:"\f15c";}
-.icon-sort-by-alphabet:before{content:"\f15d";}
-.icon-sort-by-alphabet-alt:before{content:"\f15e";}
-.icon-sort-by-attributes:before{content:"\f160";}
-.icon-sort-by-attributes-alt:before{content:"\f161";}
-.icon-sort-by-order:before{content:"\f162";}
-.icon-sort-by-order-alt:before{content:"\f163";}
-.icon-thumbs-up:before{content:"\f164";}
-.icon-thumbs-down:before{content:"\f165";}
-.icon-youtube-sign:before{content:"\f166";}
-.icon-youtube:before{content:"\f167";}
-.icon-xing:before{content:"\f168";}
-.icon-xing-sign:before{content:"\f169";}
-.icon-youtube-play:before{content:"\f16a";}
-.icon-dropbox:before{content:"\f16b";}
-.icon-stackexchange:before{content:"\f16c";}
-.icon-instagram:before{content:"\f16d";}
-.icon-flickr:before{content:"\f16e";}
-.icon-adn:before{content:"\f170";}
-.icon-bitbucket:before{content:"\f171";}
-.icon-bitbucket-sign:before{content:"\f172";}
-.icon-tumblr:before{content:"\f173";}
-.icon-tumblr-sign:before{content:"\f174";}
-.icon-long-arrow-down:before{content:"\f175";}
-.icon-long-arrow-up:before{content:"\f176";}
-.icon-long-arrow-left:before{content:"\f177";}
-.icon-long-arrow-right:before{content:"\f178";}
-.icon-apple:before{content:"\f179";}
-.icon-windows:before{content:"\f17a";}
-.icon-android:before{content:"\f17b";}
-.icon-linux:before{content:"\f17c";}
-.icon-dribbble:before{content:"\f17d";}
-.icon-skype:before{content:"\f17e";}
-.icon-foursquare:before{content:"\f180";}
-.icon-trello:before{content:"\f181";}
-.icon-female:before{content:"\f182";}
-.icon-male:before{content:"\f183";}
-.icon-gittip:before{content:"\f184";}
-.icon-sun:before{content:"\f185";}
-.icon-moon:before{content:"\f186";}
-.icon-archive:before{content:"\f187";}
-.icon-bug:before{content:"\f188";}
-.icon-vk:before{content:"\f189";}
-.icon-weibo:before{content:"\f18a";}
-.icon-renren:before{content:"\f18b";}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/content/css/geode-site.css
----------------------------------------------------------------------
diff --git a/gemfire-site/content/css/geode-site.css b/gemfire-site/content/css/geode-site.css
deleted file mode 100644
index 88bcc48..0000000
--- a/gemfire-site/content/css/geode-site.css
+++ /dev/null
@@ -1,1554 +0,0 @@
-@import url("http://fonts.googleapis.com/css?family=Open+Sans:400,300,600,800,700,400italic,600italic,700italic,800italic,300italic");
-/* @import url("http://fonts.googleapis.com/css?family=Source+Sans+Pro:200,300,400,700"); */
-/* Key scaffolding
--------------------------------------------------- */
-body {
-  position: relative; /* For scrollyspy */
-  padding-top: 50px; /* Account for fixed navbar */
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif !important;
-  font-size:14px;
-  background-color: #FAFBFC;
-  /*font-family: "Source Sans Pro", "Helvetica Neue", Helvetica, Arial, sans-serif; */
-}
-h1, h2, h3, h4, .h1, .h2, .h3, .h4 {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif !important;
-  font-weight: 300;
-}
-a:hover {text-decoration: none;}
-h2>a, h3>a, h2>a:hover, h3>a:hover { color: rgb(51,51,51); }
-h2:hover>a:after {
-  content: "\F0C1";
-  font-family: 'FontAwesome';
-  /*position:absolute;*/
-  padding-left: 12px;
-  vertical-align: 15%;
-  font-size: 20px;
-}
-h3:hover>a:after {
-  content: "\F0C1";
-  font-family: 'FontAwesome';
-  /*position:absolute;*/
-  padding-left: 8px;
-  vertical-align: 15%;
-  font-size: 15px;
-}
-h5, h6, .h5, .h6 {
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif !important;
-}
-.bf-sidebar .nav-list code{
-  margin-left: -3px;
-}
-.done {
-  opacity: 0.6;
-}
-.nav-list > li {
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 400 !important;
-  opacity: 0.8;
-  margin-bottom: 3px;
-}
-header .navbar-nav {
-  font-size: 120%;
-}
-.nav-list > li > ul {
-  padding-left: 0;
-}
-.nav-list > li ul ul {
-  padding-left: 19px;
-}
-.nav-list > li ul > li {
-  list-style-type: none !important;
-}
-.nav-list > li ul > li:before {
-  content:"\221F";
-  font-weight: 200;
-  opacity: 0.5;
-  line-height: 50%;
-  vertical-align: 60%;
-  padding-left: 3px;
-}
-.nav-list > li ul > li > a {
-  padding: 0 0 0 3px;
-  margin-bottom: 0px;
-}
-.nav-list > li > a {
-  padding: 0;
-}
-.nav-list > li > a.icns-apple:before {
-  text-align: right;
-  content: "\F179";
-  font-family: 'FontAwesome';
-  font-size: 110%;
-  margin-left: -18px;
-  padding-right: 8px;
-  line-height: 0;
-}
-.nav-list > li > a.icns-android:before {
-  text-align: right;
-  content: "\F17B";
-  font-family: 'FontAwesome';
-  font-size: 110%;
-  margin-left: -18px;
-  padding-right: 8px;
-  line-height: 0;
-}
-.nav-list > li > a.icns-html5:before {
-  text-align: right;
-  content: "\F13B";
-  font-family: 'FontAwesome';
-  font-size: 110%;
-  margin-left: -18px;
-  padding-right: 8px;
-  line-height: 0;
-}
-.nav-list > li > a.icns-windows:before {
-  text-align: right;
-  content: "\F17A";
-  font-family: 'FontAwesome';
-  margin-left: -19px;
-  padding-right: 7px;
-  line-height: 0;
-}
-li.nav-header {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 600 !important;
-  opacity: 1.0;
-  padding-top: 0 !important;
-  padding-left: 0 !important;
-  font-size: 120%;
-  margin-top: 33px;
-}
-.twitter-timeline {
-  margin-top: 33px;
-}
-.bf-docs-container .nav-list li.nav-header {
-  padding: 0 !important;
-  border-bottom: solid 1px #ccc;
-  line-height: 0;
-  margin-top: 33px;
-  margin-bottom: 13px;
-}
-.bf-docs-container .nav-list li.nav-header span {
-  background-color: #FAFBFC;
-  padding-right: 8px;
-}
-
-.bf-community h2, .bf-docs-container h2, .bf-docs-container h3 {
-  padding-top: 90px;
-  margin-top: -60px;
-}
-
-.bf-docs-container .github-callout {
-  padding: 20px 0 40px 0;
-  font-style: italic;
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  opacity: 0.8;
-  font-size: 110%;
-}
-.navbar-nav > li a {
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 300 !important;
-  opacity: 0.8;
-}
-
-.navbar-nav > li > a.major {
-  font-weight: 400 !important;
-  opacity: 1.0;
-}
-
-.navbar-collapse {
-    max-height: none;
-}
-
-.bf-masthead-bg p {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-style: normal;
-  font-weight: 200;
-  font-size: 120%;
-  line-height: 120%;
-}
-.bf-features p {
-  font-family: 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-style: normal;
-  font-weight: 400;
-}
-.bf-features p a {
-  /*font-weight: 600;*/
-  /*color: rgb(52,73,94);*/
-}
-a.btn {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 500 !important;
-  font-style: normal;
-  letter-spacing: 110%;
-}
-h1 {
-  font-weight: 200;
-  letter-spacing: 110%;
-  font-size: 200%;
-}
-.page-header {
-  padding-top: 2px;
-}
-.page-header h1 {
-  padding-top:20px;
-  font-weight: 400;
-  font-size:350%;
-}
-.page-header small{
-  float: right;
-  background-color: #fff;
-  padding-left: 10px;
-}
-.page-header small a {
-  color: #ddd;
-  font-style: italic;
-}
-.page-header small a:hover {
-  color: #2481A6;
-}
-h2 {
-  font-weight: 400;
-}
-h3 {
-  padding-top: 20px;
-  font-weight: 400 !important;
-}
-.bf-features h3 {
-  padding-top: 0;
-}
-#copyright {
-  font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-  font-weight: 300 !important;
-  font-style: italic;
-}
-#copyright img {
-  border-radius: 4px;
-}
-#copyright p {
-  opacity: 0.8;
-}
-#copyright .credits {
-  color: #fff;
-  font-weight: 600;
-  opacity: 1 !important;
-  font-style: normal;
-}
-.table {
-  margin: 20px 0 0 0;
-  background-color: transparent;
-}
-.table thead>tr>th, .table tbody>tr>th, .table tfoot>tr>th, .table thead>tr>td, .table tbody>tr>td, .table tfoot>tr>td {
-  padding-left: 0;
-}
-.toc {
-  margin-bottom:40px;
-  /*background-color: #fff;
-  border-radius: 4px;*/
-  /*padding: 20px;*/
-}
-/*.toc > h4 {
-  padding-top: 0;
-  margin-top: 0;
-  font-size: 150%;
-  font-weight:400;
-}*/
-.toc > ul {
-  margin-bottom: 0;
-  padding-left: 20px;
-}
-section > .container > h2 {
-  color: #2481A6 !important;
-}
-div.row > div.text-left > h2 {
-  margin-top: 40px;
-  margin-bottom: 10px;
-}
-.bf-docs-container h2 {
-  font-size: 250%;
-}
-/* font-face
--------------------------------------------------- */
-@font-face {
-  font-family: 'icomoon';
-  src:url('../fonts/icomoon.eot');
-  src:url('../fonts/icomoon.eot?#iefix') format('embedded-opentype'),
-    url('../fonts/icomoon.woff') format('woff'),
-    url('../fonts/icomoon.ttf') format('truetype'),
-    url('../fonts/icomoon.svg#icomoon') format('svg');
-  font-weight: normal;
-  font-style: normal;
-}
-[data-icon]:before {
-  font-family: 'icomoon';
-  content: attr(data-icon);
-  speak: none;
-  font-weight: normal;
-  font-variant: normal;
-  text-transform: none;
-  line-height: 1;
-  -webkit-font-smoothing: antialiased;
-}
-.icon-equalizer, .icon-phone-portrait, .icon-rocket, .icon-github, .icon-twitter, .icon-thumbf-up, .icon-earth, .icon-bug, .icon-wiki, .icon-releases,.icon-window {
-  font-family: 'icomoon';
-  speak: none;
-  font-style: normal;
-  font-weight: normal;
-  font-variant: normal;
-  text-transform: none;
-  line-height: 1;
-  -webkit-font-smoothing: antialiased;
-}
-.icon-equalizer:before {
-  content: "\e000";
-}
-.icon-phone-portrait:before {
-  content: "\e002";
-}
-.icon-rocket:before {
-  content: "\e003";
-}
-.icon-github:before {
-  content: "\e001";
-}
-.icon-twitter:before {
-  content: "\e004";
-}
-.icon-thumbf-up:before {
-  content: "\e005";
-}
-.icon-earth:before {
-  content: "\e006";
-}
-.icon-bug:before {
-  content: "\e007";
-}
-.icon-window:before {
-  content: "\e008";
-}
-.icon-wiki:before {
-  content: "\e009";
-}
-.icon-releases:before {
-  content: "\e010";
-}
-/* Custom docs button */
-.btn-bf {
-  color: #563d7c;
-  background-color: #fff;
-  border-color: #e5e5e5;
-}
-.btn-bf:hover,
-.btn-bf:focus,
-.btn-bf:active {
-  color: #fff;
-  background-color: #563d7c;
-  border-color: #563d7c;
-}
-/* Navs and headers
--------------------------------------------------- */
-header.navbar .container {
-  padding-right: 0;
-}
-footer > .container {
-  padding-right: 0;
-  padding-left: 0;
-}
-header.navbar.secondary {
-  box-shadow: 0px 1px 2px #7b48fe;
-}
-.navbar-inverse {
-    background-color: #3c3c3c;
-    /*background-color: #3c3c3c;*/
-}
-nav.navbar-collapse {
-  border-color: rgb(52, 73, 94) !important;
-  border-width: 1px;
-  box-shadow: none;
-  padding-left: 15px;
-}
-.bf-docs-nav {
-  border-bottom: 0px;
-  padding-left: 20px;
-  padding-right: 20px;
-}
-.navbar-nav {
-  margin: 0;
-}
-
-.navbar-brand {
-  padding-left: 0;
-}
-.bf-docs-nav .navbar-nav > li > a {
-  margin: 10px 0 0 10px;
-  padding: 5px;
-  -webkit-border-radius: 4px;
-  -moz-border-radius: 4px;
-  border-radius: 4px;
-  color: white;
-}
-.bf-docs-nav .navbar-nav > li > a:hover {
-  color: #fff;
-  background-color: #1e90ff;
-}
-.bf-docs-nav .navbar-nav > .active > a,
-.bf-docs-nav .navbar-nav > .active > a:hover {
-  color: #fff;
-  background-color: #1e90ff !important;
-}
-li.divider {
-  margin-left: 0 !important;
-}
-
-/* Homepage masthead
--------------------------------------------------- */
-.bf-masthead {
-  position: relative;
-  text-align: left;
-  /*background: #cbcbcd url(../img/rende.png) no-repeat 50% 100px;*/
-}
-.bf-masthead .bf-masthead-bg {
-    /*margin-top: 10px;*/
-    padding: 30px 15px;
-    color:#3c3c3c;
-    background-color: #fff;
-}
-/*.bf-masthead .logo {
-  height: 203px;
-}*/
-.bf-masthead .logo-title {
-  width: 600px;
-  padding: 20px 0 0 0;
-  margin-left: 0;
-}
-.bf-masthead .text-container {
-  margin-left: auto;
-  margin-right: auto;
-  text-align: left;
-}
-.bf-masthead .tagline {
-  font-style:normal;
-  font-weight:200;
-  font-size: 300%;
-  margin-bottom: 0;
-  margin-top: 30px;
-  padding-bottom: 30px;
-  color: #3c3c3c;
-  line-height: 90%;
-}
-.bf-masthead .tagline em {
-  font-style: italic;
-}
-.bf-masthead .description {
-  font-style: normal;
-  font-size: 150%;
-  margin-bottom: 45px;
-  /*line-height: 100%;*/
-}
-.bf-masthead .btn-wrapper {
-  text-align: center;
-  margin-bottom: 20px;
-  clear:both;
-}
-.bf-masthead .btn-wrapper {
-  text-align: left;
-  width: 100%;
-  margin-right: 20px;
-}
-.bf-masthead .social-btn {
-  float: left;
-  margin: 0 20px 10px 0;
-}
-.bf-masthead .or {
-  margin-left: 0px;
-  font-style: normal;
-  font-weight: 200;
-  opacity: 0.8;
-}
-
-.bf-masthead a {
-  line-height: 30px;
-  color: #FAFBFC;
-  font-style: normal;
-  font-weight: 400;
-}
-.bf-masthead h1 {
-  font-size: 30px;
-  line-height: 1.4;
-  color: #fff;
-}
-.bf-masthead .bf-masthead-links a,
-.bf-masthead .bf-masthead-links a:hover {
-  color: #fff;
-}
-.bf-masthead-info {
-  padding: 80px 20px;
-  font-size: 28px;
-  text-align: center;
-  background-color: #fc5d7b;
-  color: rgba(0, 0, 0, .3)
-}
-.bf-masthead-info .icon-angle-down {
-  font-size: 40px;
-}
-
-/* Download button */
-.bf-masthead .btn-outline {
-  margin-top: 20px;
-  margin-bottom: 20px;
-  border: 1px solid #fff;
-  padding: 18px 24px;
-  font-size: 21px;
-  color: #fff;
-}
-.bf-masthead .btn-outline:hover {
-  background-color: #fff;
-  color: #42a8d2;
-}
-
-.bf-masthead-links {
-  margin-top: 20px;
-  margin-bottom: 20px;
-  padding-left: 0;
-  list-style: none;
-  text-align: center;
-}
-.bf-masthead-links li {
-  margin-bottom: 15px;
-  color: #fff;
-}
-.bf-masthead-links a {
-  color: #c8c7c6;
-}
-
-/* Homepage features
--------------------------------------------------- */
-section > .container > h2 {
-  text-align: left;
-  margin-bottom: 16px;
-  padding-bottom: 6px;
-  font-size: 28px;
-  color:#3c3c3c;
-  /* border-bottom: 1px solid #e5e5e5; */
-}
-.bf-features {
-  color: white;
-  text-align: center;
-  padding: 20px 0;
-}
-.bf-features h2 ~ p {
-  margin-bottom: 40px;
-  font-size: 16px;
-}
-.bf-features .bf-icon-wrap {
-  margin: 0 auto 10px;
-  -webkit-border-radius: 50%;
-  -moz-border-radius: 50%;
-  border-radius: 50%;
-  background-color: #7b48fe;
-  width: 90px;
-  height: 90px;
-  line-height: 90px;
-  font-size: 50px;
-  color: #FAFBFC;
-}
-.bf-features h3 {
-  margin: 0 0 10px;
-  color: rgb(52,73,94);
-  font-weight: 400 !important;
-}
-.bf-features h3 ~ p {
-  color: #656565;
-}
-/*
-.bf-features2 {
-  color: #292929;
-  text-align: center;
-}
-.bf-features2 {
-  padding: 40px 0;
-}
-.bf-features2 h2 {
-  text-align: left;
-  margin-bottom: 16px;
-  padding-bottom: 6px;
-  font-size: 28px;
-  color:#666666;
-  border-bottom: 1px solid #e5e5e5;
-}
-.bf-features2 h2 ~ p {
-  margin-bottom: 40px;
-  font-size: 16px;
-}
-.bf-features2 .bf-icon-wrap {
-  margin: 0 auto 20px;
-  -webkit-border-radius: 50%;
-  -moz-border-radius: 50%;
-  border-radius: 50%;
-  background-color: #e5e9ed;
-  width: 80px;
-  height: 80px;
-  line-height: 80px;
-  font-size: 40px;
-  color: #a5b6c7;
-}
-.bf-features2 h3 {
-  margin: 0 0 10px;
-  color: #696969;
-}
-.bf-features2 h3 ~ p {
-  color: #656565;
-} */
-.bf-features-inverse {
-  padding: 140px 0;
-  font-size: 16px;
-  color: white;
-  text-align: center;
-  background-color: #3c3c3c;
-}
-.bf-features-inverse h2 {
-  margin-bottom: 20px;
-  font-size: 40px;
-}
-.bf-features-inverse h2 ~ p {
-  margin-bottom: 40px;
-  font-size: 16px;
-}
-.bf-features-inverse .bf-icon-wrap {
-  margin: 0 auto 20px;
-  -webkit-border-radius: 50%;
-  -moz-border-radius: 50%;
-  border-radius: 50%;
-  background-color: #fff;
-  width: 80px;
-  height: 80px;
-  line-height: 80px;
-  font-size: 40px;
-  color: #3dc0f1;
-}
-.bf-features-inverse h3 ~ p {
-  font-size: 14px;
-}
-.bf-features-inverse a,
-.bf-features-inverse a:hover {
-  color: #fff;
-  text-decoration: underline;
-}
-.bf-news ul {
-  margin-bottom: 20px;
-}
-
-.bf-questions .container {
-  border-top: 1px solid rgb(52,73,94);
-  border-bottom: 1px solid rgb(52,73,94);
-  padding: 60px 0;
-  margin-top: 0px;
-}
-
-.bf-questions div {
-  line-height: 30px;
-}
-
-.bf-community .container, .bf-tagline .container {
-  padding-left: 0;
-  padding-right: 0;
-}
-
-.bf-community h3 {
-  font-weight: 600;
-  margin-top: 0;
-}
-
-.bf-community .row a p {
-  color: rgb(51,51,51);
-}
-
-/*.bf-community h2 {
-  padding-bottom: 10px;
-  border-bottom: 1px solid rgb(51,51,51);
-}*/
-
-.bf-community h2 {
-  /*padding: 0 !important;*/
-  border-bottom: solid 1px #ccc;
-  line-height: 0;
-  /*margin-top: 33px;*/
-  margin-bottom: 13px;
-}
-.bf-releases h2 {
-  /*padding: 0 !important;*/
-  border-bottom: solid 1px #ccc;
-  line-height: 0;
-  /*margin-top: 33px;*/
-  margin-bottom: 13px;
-}
-
-.bf-community h2 span {
-  background-color: #FAFBFC;
-  padding-right: 10px;
-}
-
-
-.bf-releases h2 span {
-  background-color: #FAFBFC;
-  padding-right: 10px;
-}
-
-.bf-releases p {
-  padding: 10px 0 5px 0;
-}
-
-.bf-community {
-  margin-top: 80px;
-}
-
-.bf-releases {
-  margin-top: 10px;
-}
-
-#map-canvas {
-  border-bottom: 1px solid rgb(52, 73, 94);
-}
-
-.bf-community img {
-  border-radius: 4px;
-  border: solid 1px rgb(52, 73, 94);
-  margin-top: 20px;
-  margin-bottom: 10px;
-  width: 100%;
-  box-shadow: rgb(52, 73, 94) 2px 2px 4px;
-}
-
-.bf-releases img {
-  border-radius: 4px;
-  border: solid 1px rgb(52, 73, 94);
-  margin-top: 20px;
-  margin-bottom: 10px;
-  width: 100%;
-  box-shadow: rgb(52, 73, 94) 2px 2px 4px;
-}
-
-
-h2.icns-calendar:before{
-  text-align: right;
-  content: "\F073";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -40px;
-  padding-right: 10px;
-  line-height: 0;
-}
-h2.icns-chat:before{
-  text-align: right;
-  content: "\F0FC";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -40px;
-  padding-right: 10px;
-  line-height: 0;
-}
-
-h2.icns-envelope:before{
-  text-align: right;
-  content: "\F0E0";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -40px;
-  padding-right: 10px;
-  line-height: 0;
-}
-h2.icns-briefcase:before{
-  text-align: right;
-  content: "\F0B1";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -42px;
-  padding-right: 10px;
-  line-height: 0;
-}
-h2.icns-group:before{
-  text-align: right;
-  content: "\F005";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -37px;
-  padding-right: 8px;
-  line-height: 0;
-}
-h2.icns-tools:before{
-  text-align: right;
-  content: "\F006";
-  font-family: 'FontAwesome';
-  font-size: 100%;
-  margin-left: -37px;
-  padding-right: 8px;
-  line-height: 0;
-}
-/* Docs pages and sections
--------------------------------------------------- */
-
-/* Page headers */
-.bf-header {
-  padding: 5px 20px 10px;
-  font-size: 16px;
-  color: #5a5a5a;
-  text-align: center;
-  border-bottom: 1px solid #e5e5e5;
-}
-.bf-header p {
-  font-weight: 300;
-  line-height: 1.5;
-}
-.bf-header .container {
-  position: relative;
-}
-
-.bf-docs-container {
-  margin-top: 60px;
-}
-
-.bf-docs-container .col-md-3{
-  padding-left: 0;
-}
-
-
-.bf-docs-container .col-md-9.main-article {
-  background-color: #fff;
-  border-radius: 4px;
-  border: 1px solid #ccc;
-  padding-bottom: 15px;
-  min-height: 820px;
-}
-
-.bf-docs-section + .bf-docs-section {
-  padding-top: 20px;
-}
-
-.bf-docs-container .nav-list {
-  padding-top: 20px;
-}
-
-/* Bootstrap code examples
--------------------------------------------------- */
-
-/* Base class */
-.bf-example {
-  position: relative;
-  margin: 20px 0 40px;
-}
-.bf-example > .btn-toolbar + .btn-toolbar {
-  margin-top: 10px;
-}
-
-.bf-example .row {
-  margin-bottom: 20px;
-}
-
-/* Tweak display of the examples */
-.bf-example + .highlight {
-  margin: 0 -15px;
-  border-radius: 0;
-  border-width: 0 0 1px;
-}
-
-/* Tweak content of examples for optimum awesome */
-.bf-example > p:last-child,
-.bf-example > ul:last-child,
-.bf-example > ol:last-child,
-.bf-example > blockquote:last-child,
-.bf-example > input:last-child,
-.bf-example > select:last-child,
-.bf-example > textarea:last-child,
-.bf-example > .table:last-child,
-.bf-example > .navbar:last-child
-.bf-example > .jumbotron:last-child,
-.bf-example > .alert:last-child,
-.bf-example > .panel:last-child,
-.bf-example > .list-group:last-child,
-.bf-example > .well:last-child {
-  margin-bottom: 0;
-}
-.bf-example > p > .close {
-  float: none;
-}
-.bf-example-title {
-  border-left: 3px solid #08d;
-  padding-left: 10px;
-  line-height: 24px;
-  margin: 20px 0;
-  font-size: 16px;
-}
-.bf-example-align .btn {
-  margin-top: 4px;
-}
-.bf-example-align .btn-xs {
-  margin-top: 16px;
-}
-.bf-example-align .btn-sm {
-  margin-top: 8px;
-}
-.bf-example-align .btn-lg {
-  margin-top: -7px;
-}
-
-/* Navbar examples */
-.bf-example .navbar:last-child {
-  margin-bottom: 0;
-}
-.bf-navbar-top-example,
-.bf-navbar-bottom-example {
-  z-index: 1;
-  padding: 0;
-  min-height: 110px;
-  overflow: hidden; /* cut the drop shadows off */
-}
-.bf-navbar-top-example .navbar-fixed-top,
-.bf-navbar-bottom-example .navbar-fixed-bottom {
-  position: relative;
-  margin-left: 0;
-  margin-right: 0;
-}
-.bf-navbar-top-example .navbar-fixed-top {
-  top: -1px;
-}
-.bf-navbar-bottom-example .navbar-fixed-bottom {
-  bottom: -1px;
-}
-.bf-navbar-top-example {
-  -webkit-border-radius: 0 0 4px 4px;
-     -moz-border-radius: 0 0 4px 4px;
-          border-radius: 0 0 4px 4px;
-}
-.bf-navbar-top-example:after {
-  top: auto;
-  bottom: 15px;
-  -webkit-border-radius: 0 4px 0 4px;
-     -moz-border-radius: 0 4px 0 4px;
-          border-radius: 0 4px 0 4px;
-}
-.bf-navbar-bottom-example {
-  -webkit-border-radius: 4px 4px 0 0;
-     -moz-border-radius: 4px 4px 0 0;
-          border-radius: 4px 4px 0 0;
-}
-.bf-navbar-bottom-example .navbar {
-  margin-bottom: 0;
-}
-.bf-navbar-top-example.bf-example:after {
-  bottom: -1px;
-  border-radius: 0 4px;
-}
-
-/* Example modals */
-.bf-example-modal {
-  background-color: #f5f5f5;
-}
-.bf-example-modal .modal {
-  position: relative;
-  top: auto;
-  right: auto;
-  left: auto;
-  bottom: auto;
-  z-index: 1;
-  display: block;
-}
-.bf-example-modal .modal-dialog {
-  left: auto;
-  margin-left: auto;
-  margin-right: auto;
-}
-
-/* Example dropdowns */
-.bf-example > .dropdown > .dropdown-menu,
-.bf-example-submenu > .pull-left > .dropup > .dropdown-menu,
-.bf-example-submenu > .pull-left > .dropdown > .dropdown-menu {
-  position: static;
-  display: block;
-  margin-bottom: 5px;
-}
-.bf-example-submenu {
-  min-height: 230px;
-}
-.bf-example-submenu > .pull-left + .pull-left {
-  margin-left: 20px;
-}
-
-/* Tooltips */
-.bf-example-tooltips {
-  text-align: center;
-}
-
-/* Popovers */
-.bf-example-popover {
-  padding-bottom: 24px;
-}
-.bf-example-popover .popover {
-  position: relative;
-  display: block;
-  float: left;
-  width: 260px;
-  margin: 20px;
-}
-
-/* Example templates
--------------------------------------------------- */
-
-.bf-examples h4 {
-  margin-bottom: 5px;
-}
-.bf-examples p {
-  margin-bottom: 20px;
-}
-
-/* Responsive docs
--------------------------------------------------- */
-
-/* Responsive (scrollable) doc tables */
-@media (max-width: 768px) {
-  .bf-table-scrollable {
-    width: 100%;
-    margin-bottom: 15px;
-    overflow-y: hidden;
-    overflow-x: scroll;
-    border: 1px solid #ddd;
-  }
-  .bf-table-scrollable .table {
-    margin-bottom: 0;
-    border: 0;
-  }
-  .bf-table-scrollable .table th,
-  .bf-table-scrollable .table td {
-    white-space: nowrap;
-  }
-  .bf-table-scrollable .table th:first-child,
-  .bf-table-scrollable .table td:first-child {
-    border-left: 0;
-  }
-  .bf-table-scrollable .table th:last-child,
-  .bf-table-scrollable .table td:last-child {
-    border-right: 0;
-  }
-  .bf-table-scrollable .table tr:last-child th,
-  .bf-table-scrollable .table tr:last-child td {
-    border-bottom: 0;
-  }
-}
-
-/* Footer
--------------------------------------------------- */
-
-.bf-footer {
-  margin-top: 40px;
-  padding: 30px 5px;
-  /*text-align: left;*/
-  background:#1e90ff;
-  color:white;
-}
-
-.bf-footer a {
-    color:#f0f0f0 !important;
-}
-.bf-footer.bf-footer-margin {
-    margin-top: 100px;
-    margin-bottom: 20px;
-}
-
-.bf-footer .bf-icon-wrap {
-    font-size: 40px;
-    color: #a5b6c7;
-    display: inline;
-    margin: 10px;
-}
-
-#footer-icons {
-    float: right;
-    display: inline;
-    line-height: 20px;
-    margin: 40px 10px 20px 10px;
-}
-
-#copyright {
-    float: left;
-    display: inline;
-    line-height: 20px;
-    margin: 20px 10px 20px 10px;
-}
-
-/* Social proof buttons from GitHub & Twitter */
-.bf-social {
-  margin-bottom: 20px;
-}
-.bf-social-buttons {
-  display: inline-block;
-  margin-bottom: 0;
-  padding-left: 0;
-  list-style: none;
-}
-.bf-social-buttons li {
-  display: inline-block;
-  line-height: 1;
-}
-.bf-social-buttons li + li {
-  margin-left: 15px;
-}
-.bf-social-buttons .twitter-follow-button {
-  width: 225px !important;
-}
-.bf-social-buttons .twitter-share-button {
-  width: 98px !important;
-}
-/* Style the GitHub buttons via CSS instead of inline attributes */
-.github-btn {
-  border: 0;
-  overflow: hidden;
-}
-
-.highlight {
-  padding: 9px 14px;
-  margin-bottom: 40px;
-  border: 1px solid #e1e1e8;
-  border-radius: 4px;
-}
-.highlight pre {
-  padding: 0;
-  margin-top: 0;
-  margin-bottom: 0;
-  background-color: rgb(248, 248, 255);
-  border: 0;
-  white-space: nowrap;
-}
-.highlight pre code {
-  font-size: inherit;
-  font-family: Consolas, 'Liberation Mono', Courier, monospace;
-  color: #333; /* Effectively the base text color */
-}
-.highlight pre .lineno {
-  display: inline-block;
-  width: 22px;
-  padding-right: 5px;
-  margin-right: 10px;
-  text-align: right;
-  color: #bebec5;
-}
-
-.c { color: #999988; font-style: italic } /* Comment */
-.err { color: #a61717; background-color: #e3d2d2 } /* Error */
-.k { color: #000000; font-weight: bold } /* Keyword */
-.o { color: #000000; font-weight: bold } /* Operator */
-.cm { color: #999988; font-style: italic } /* Comment.Multiline */
-.cp { color: #999999; font-weight: bold; font-style: italic } /* Comment.Preproc */
-.c1 { color: #999988; font-style: italic } /* Comment.Single */
-.cs { color: #999999; font-weight: bold; font-style: italic } /* Comment.Special */
-.gd { color: #000000; background-color: #ffdddd } /* Generic.Deleted */
-.ge { color: #000000; font-style: italic } /* Generic.Emph */
-.gr { color: #aa0000 } /* Generic.Error */
-.gh { color: #999999 } /* Generic.Heading */
-.gi { color: #000000; background-color: #ddffdd } /* Generic.Inserted */
-.go { color: #888888 } /* Generic.Output */
-.gp { color: #555555 } /* Generic.Prompt */
-.gs { font-weight: bold } /* Generic.Strong */
-.gu { color: #aaaaaa } /* Generic.Subheading */
-.gt { color: #aa0000 } /* Generic.Traceback */
-.kc { color: #000000; font-weight: bold } /* Keyword.Constant */
-.kd { color: #000000; font-weight: bold } /* Keyword.Declaration */
-.kn { color: #000000; font-weight: bold } /* Keyword.Namespace */
-.kp { color: #000000; font-weight: bold } /* Keyword.Pseudo */
-.kr { color: #000000; font-weight: bold } /* Keyword.Reserved */
-.kt { color: #445588; font-weight: bold } /* Keyword.Type */
-.m { color: #009999 } /* Literal.Number */
-.s { color: #d01040 } /* Literal.String */
-.na { color: #008080 } /* Name.Attribute */
-.nb { color: #0086B3 } /* Name.Builtin */
-.nc { color: #445588; font-weight: bold } /* Name.Class */
-.no { color: #008080 } /* Name.Constant */
-.nd { color: #3c5d5d; font-weight: bold } /* Name.Decorator */
-.ni { color: #800080 } /* Name.Entity */
-.ne { color: #990000; font-weight: bold } /* Name.Exception */
-.nf { color: #990000; font-weight: bold } /* Name.Function */
-.nl { color: #990000; font-weight: bold } /* Name.Label */
-.nn { color: #555555 } /* Name.Namespace */
-.nt { color: #000080 } /* Name.Tag */
-.nv { color: #008080 } /* Name.Variable */
-.ow { color: #000000; font-weight: bold } /* Operator.Word */
-.w { color: #bbbbbb } /* Text.Whitespace */
-.mf { color: #009999 } /* Literal.Number.Float */
-.mh { color: #009999 } /* Literal.Number.Hex */
-.mi { color: #009999 } /* Literal.Number.Integer */
-.mo { color: #009999 } /* Literal.Number.Oct */
-.sb { color: #d01040 } /* Literal.String.Backtick */
-.sc { color: #d01040 } /* Literal.String.Char */
-.sd { color: #d01040 } /* Literal.String.Doc */
-.s2 { color: #d01040 } /* Literal.String.Double */
-.se { color: #d01040 } /* Literal.String.Escape */
-.sh { color: #d01040 } /* Literal.String.Heredoc */
-.si { color: #d01040 } /* Literal.String.Interpol */
-.sx { color: #d01040 } /* Literal.String.Other */
-.sr { color: #009926 } /* Literal.String.Regex */
-.s1 { color: #d01040 } /* Literal.String.Single */
-.ss { color: #990073 } /* Literal.String.Symbol */
-.bp { color: #999999 } /* Name.Builtin.Pseudo */
-.vc { color: #008080 } /* Name.Variable.Class */
-.vg { color: #008080 } /* Name.Variable.Global */
-.vi { color: #008080 } /* Name.Variable.Instance */
-.il { color: #009999 } /* Literal.Number.Integer.Long */
-
-.css .o,
-.css .o + .nt,
-.css .nt + .nt { color: #999; }
-/* Show code snippets when we have the space */
-@media screen and (min-width: 481px) {
-  .highlight {
-    display: block;
-  }
-}
-
-.demoColorSwatches {
-    color: #FFF;
-}
-.demoColorSwatches h4 {
-    color: #292929;
-}
-.demoColorSwatches .colorPanel {
-    padding: 15px;
-}
-.demoColorSwatches .colorPanel:hover {
-    opacity: 0.85;
-    filter: alpha(opacity=85);
-}
-
-/* Responsive variations
--------------------------------------------------- */
-
-/* Hide code snippets on mobile devices */
-@media screen and (max-width: 480px) {
-  .highlight {
-    display: none;
-  }
-}
-
-
-.bf-docs-container {
-    margin-top: 40px;
-}
-
-/* Navbars */
-.bf-example .navbar:last-child {
-  margin-bottom: 0;
-}
-.bf-navbar-top-example,
-.bf-navbar-bottom-example {
-  z-index: 1;
-  padding: 0;
-  overflow: hidden; /* cut the drop shadows off */
-  border: 1px solid #ddd ;
-}
-.bf-navbar-top-example .navbar-header,
-.bf-navbar-bottom-example .navbar-header {
-  margin-left: 0;
-}
-.bf-navbar-top-example .navbar-fixed-top,
-.bf-navbar-bottom-example .navbar-fixed-bottom {
-  position: relative;
-  margin-left: 0;
-  margin-right: 0;
-}
-.bf-navbar-top-example {
-  padding-bottom: 45px;
-}
-.bf-navbar-top-example:after {
-  top: auto;
-  bottom: 15px;
-}
-.bf-navbar-top-example .navbar-fixed-top {
-  top: -1px;
-}
-.bf-navbar-bottom-example {
-  padding-top: 65px;
-}
-.bf-navbar-bottom-example .navbar-fixed-bottom {
-  bottom: -1px;
-}
-.bf-navbar-bottom-example .navbar {
-  margin-bottom: 0;
-}
-
-/* Phone and below */
-@media screen and (max-width: 768px) {
-  #home-logo { opacity: 1; height: 25px;}
-  span.icon-book:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Docs";
-  }
-  span.icon-group:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Community";
-  }
-  span.icon-github-sign:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Github";
-  }
-  span.icon-trello:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Trello";
-  }
-  span.icon-bug:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " JIRA";
-  }
-  span.icon-stackexchange:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " StackOverflow";
-  }
-  span.icon-chat:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " HipChat";
-  }
-  span.icon-twitter:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Twitter";
-  }
-  span.icon-edit:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: "  Wiki";
-  }
-  span.icon-releases:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Releases";
-  }
-  span.icon-comments:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Chat";
-  }
-  span.icon-envelope:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Mailing Lists";
-  }
-  div.cta {
-    line-height: 60px;
-  }
-  span.avoidwrap { display:inline-block; }
-  .navbar-nav > li > a {
-    opacity: 1.0;
-  }
-  .bf-masthead .tagline {
-    line-height: 90%;
-  }
-
-  .bf-masthead .description {
-    margin-bottom: 30px;
-  }
-  .bf-masthead p {
-    clear:both;
-    line-height: 100%;
-    padding-top: 5px;
-  }
-  .bf-masthead .or {
-    margin-right: 0;
-  }
-  .bf-questions .container{
-    padding: 30px 0;
-  }
-}
-
-/* Phones and Tablets */
-/*@media screen and (min-width: 768px) and (max-width: 992px) {
-
-  .bf-masthead .tagline {
-    padding-bottom: 20px;
-  }
-
-}*/
-
-@media screen and (max-width: 992px) {
-  .bf-community.committers .col-md-3 p{
-    margin-bottom: 40px;
-  }
-}
-
-/* Tablets and below */
-@media screen and (max-width: 1200px) {
-  .row {
-    padding: 0 25px;
-  }
-  .bf-docs-container .nav-list {
-    padding-left: 30px;
-  }
-  .bf-docs-container .colb-md-3 .nav-list {
-    padding-top: 0;
-  }
-  .bf-community .container, .bf-tagline .container {
-    padding-left: 50px;
-  }
-  .bf-community.committers .col-md-3 h3{
-    margin-top: 0px;
-  }
-  #map-canvas { height: 300px; margin-top: 5px;}
-}
-/* Tablets and up */
-@media screen and (min-width: 768px) {
-  header.secondary #home-logo { opacity: 1; }
-  #home-logo { opacity: 0; height: 40px;}
-  .bf-header {
-    font-size: 21px;
-    text-align: left;
-  }
-  .bf-header h1 {
-    font-size: 60px;
-    line-height: 1;
-  }
-
-  .bf-masthead p {
-    float:left;
-  }
-
-  .bf-masthead .or {
-    margin-right: 20px;
-  }
-  .bf-features .row {
-    padding: 20px 0;
-  }
-  .bf-example {
-    margin-left: 0;
-    margin-right: 0;
-    border-radius: 4px;
-    background-color: #fff;
-    border-width: 1px;
-    border-color: #ddd;
-    box-shadow: none;
-  }
-  .bf-example + .prettyprint,
-  .bf-example + .highlight {
-    margin-top: -16px;
-    margin-left: 0;
-    margin-right: 0;
-    border-width: 1px;
-    border-bottom-left-radius: 4px;
-    border-bottom-right-radius: 4px;
-  }
-
-  /* Show the docs nav */
-  .bf-sidebar {
-    display: block;
-  }
-  .bf-sidebar > .nav > li > a {
-    display: inline-block;
-  }
-
-  .bf-navbar-top-example .navbar-fixed-top,
-  .bf-navbar-bottom-example .navbar-fixed-bottom {
-    position: absolute;
-  }
-  .bs-navbar-top-example {
-    border-radius: 0 0 4px 4px;
-  }
-  .bs-navbar-bottom-example {
-    border-radius: 4px 4px 0 0;
-  }
-}
-
-/* Tablets */
-@media screen and (min-width: 768px) and (max-width: 1200px) {
-  span.icns {
-    font-size: 140%;
-  }
-}
-
-/* Tablets/desktops and up */
-@media screen and (min-width: 1200px) {
-  span.icon-book:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Docs";
-  }
-  span.icon-group:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Community";
-  }
-  span.icon-github-sign:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Github";
-  }
-  span.icon-trello:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Trello";
-  }
-  span.icon-bug:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " JIRA";
-  }
-  span.icon-stackexchange:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " StackOverflow";
-  }
-  span.icon-chat:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " HipChat";
-  }
-  span.icon-twitter:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Twitter";
-  }
-  span.icon-edit:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: "  Wiki";
-  }
-  span.icon-releases:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Releases";
-  }
-  span.icon-comments:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Chat";
-  }
-  span.icon-envelope:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Mailing Lists";
-  }
-  .bf-header h1,
-  .bf-header p {
-    margin-right: 380px;
-  }
-  .bf-docs-container {
-    margin-top: 40px;
-  }
-
-  .bf-masthead .tagline, .bf-masthead .description, .bf-masthead .btn-wrapper {
-    margin-left: 120px;
-  }
-  .bf-community.committers .col-md-3 h3{
-    margin-top: 40px;
-  }
-  .bf-community.committers img {
-    width: 100%;
-  }
-  #map-canvas { height: 500px; margin-top: 10px;}
-
-}
-
-.navbar-toggle {
-  background-color: rgb(36,129,166);
-  margin: 12px 0 0 0;
-}


[15/50] [abbrv] incubator-geode git commit: GEODE-608 - Adding licence headers to gemfire-site GEODE-652 - Removing compiled website pages from gemfire-site - Added gitignore for gemfire-site

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gemfire-site/website/utilities/usergrid.csv
----------------------------------------------------------------------
diff --git a/gemfire-site/website/utilities/usergrid.csv b/gemfire-site/website/utilities/usergrid.csv
deleted file mode 100644
index 485c7e3..0000000
--- a/gemfire-site/website/utilities/usergrid.csv
+++ /dev/null
@@ -1,290 +0,0 @@
-Members for group Usergrid
-Email address,Nickname,Group status,Email status,Email preference,Posting permissions,Join year,Join month,Join day,Join hour,Join minute,Join second,Time zone
-0xfleo@gmail.com,,member,,"email","allowed",2013,6,5,18,39,55,America/Los_Angeles
-aaadonai@gmail.com,"AR",member,,"no email","allowed",2012,7,22,23,11,15,America/Los_Angeles
-aaron.allsbrook@gmail.com,"Aaron Allsbrook",member,,"no email","allowed",2013,5,14,12,53,42,America/Los_Angeles
-aaron@lingotek.com,"Aaron Davis",member,,"digest","allowed",2012,7,11,12,1,2,America/Los_Angeles
-aaustin@apigee.com,"Ashley Austin",member,,"no email","allowed",2013,6,11,11,35,17,America/Los_Angeles
-abbdef@gmail.com,"James Hancock",member,,"no email","allowed",2012,7,12,5,12,51,America/Los_Angeles
-achachiez@gmail.com,"Emmanuel Nyachoke",member,,"no email","allowed",2013,4,23,1,0,29,America/Los_Angeles
-achalil@gmail.com,"anil chalil",member,,"no email","allowed",2013,2,14,9,44,55,America/Los_Angeles
-ad2joe@gmail.com,"ad2joe",member,,"abridged","allowed",2012,9,13,0,40,46,America/Los_Angeles
-adam@dymitruk.com,"Adam Dymitruk",member,,"no email","allowed",2013,1,14,12,4,46,America/Los_Angeles
-adam@enigmaticflare.co.uk,"Adam Roberts",member,,"no email","allowed",2012,8,14,23,18,59,America/Los_Angeles
-akaia75@gmail.com,"이현태",member,,"email","allowed",2012,9,3,17,6,46,America/Los_Angeles
-akarasulu@apigee.com,"Alex Karasulu",member,,"no email","allowed",2013,9,26,11,52,54,America/Los_Angeles
-akarasulu@gmail.com,"Alex Karasulu",member,,"no email","allowed",2013,7,16,7,26,18,America/Los_Angeles
-alan.m.shelly@gmail.com,"Alan Shelly",member,,"no email","allowed",2013,3,31,23,12,53,America/Los_Angeles
-alan@apigee.com,"Alan Ho",member,,"abridged","allowed",2013,4,2,9,0,14,America/Los_Angeles
-alberto.alvine@corp.globo.com,"Alberto Leal",member,,"email","not allowed",2013,1,16,7,43,24,America/Los_Angeles
-alexandernetworks@gmail.com,"Kenneth L. Fax",member,,"no email","allowed",2013,6,21,10,48,1,America/Los_Angeles
-alexwhite3000@gmail.com,"Alex White",member,,"no email","allowed",2013,6,2,5,31,36,America/Los_Angeles
-alothien@gmail.com,"Kevin Hoffman",member,,"no email","allowed",2013,10,19,14,31,30,America/Los_Angeles
-alvaro.pereda@gmail.com,"Alvaro Pereda",member,,"no email","allowed",2012,12,7,7,12,56,America/Los_Angeles
-amir.moghimi@gmail.com,"Amir",member,,"abridged","allowed",2012,6,12,21,44,22,America/Los_Angeles
-amr.zagloul@gmail.com,"amr_zagloul",member,,"no email","allowed",2012,2,20,3,27,39,America/Los_Angeles
-amund@appdevs.no,"Amund",member,,"no email","allowed",2012,7,17,5,57,4,America/Los_Angeles
-amuramoto@apigee.com,"Alex Muramoto",member,,"abridged","allowed",2013,7,8,10,9,58,America/Los_Angeles
-angel.marquez@gmail.com,"Angel",member,,"email","allowed",2011,10,3,20,17,41,America/Los_Angeles
-anwithct@gmail.com,"anwith ct",member,,"no email","allowed",2012,10,7,23,42,19,America/Los_Angeles
-askagirl@gmail.com,"fritz lee",member,,"no email","allowed",2012,7,3,1,45,42,America/Los_Angeles
-athyholdt@gmail.com,"Andreas Thyholdt",member,,"abridged","allowed",2013,7,11,3,31,55,America/Los_Angeles
-authenticgeek@gmail.com,"Nick O'Neill",member,,"no email","allowed",2013,2,20,19,23,5,America/Los_Angeles
-avs.nagendra@gmail.com,"Nagendra addepalli",member,,"abridged","allowed",2013,6,20,23,29,59,America/Los_Angeles
-aybars.sogancilar@gmail.com,"aybars",member,,"abridged","allowed",2013,10,1,8,20,41,America/Los_Angeles
-bandara.nuwan@gmail.com,"Nuwan Bandara",member,,"no email","allowed",2013,9,20,7,14,45,America/Los_Angeles
-ben@benatkin.com,"Ben Atkin",member,,"no email","allowed",2013,3,22,14,26,18,America/Los_Angeles
-benbjammin@gmail.com,,member,,"no email","allowed",2013,3,22,14,28,9,America/Los_Angeles
-bernard.franssen@gmail.com,"Bfr",member,,"no email","allowed",2013,3,17,10,31,3,America/Los_Angeles
-bharu0503@gmail.com,"Bharathi R",member,,"no email","allowed",2013,9,4,3,44,7,America/Los_Angeles
-boudreault.alan@gmail.com,"Alan Boudreault",member,,"no email","allowed",2013,6,13,17,51,45,America/Los_Angeles
-bradjonesca@gmail.com,"Brad Jones",member,,"email","allowed",2012,7,12,6,23,33,America/Los_Angeles
-brian@brianchildress.co,,member,,"no email","allowed",2013,3,23,14,45,8,America/Los_Angeles
-broadhigh@gmail.com,"Broadhigh Computer",member,,"abridged","allowed",2013,7,11,18,34,57,America/Los_Angeles
-bryan.martinez@gmail.com,"extra_rice",member,,"digest","allowed",2012,8,7,15,46,17,America/Los_Angeles
-carchrae@gmail.com,"Tom Carchrae",member,,"no email","allowed",2012,4,6,5,48,15,America/Los_Angeles
-carlos.eberhardt@gmail.com,"Carlos Eberhardt",member,,"email","allowed",2012,5,18,19,36,30,America/Los_Angeles
-cengizbilgic85@gmail.com,"cengiz bilgic",member,,"no email","allowed",2012,9,21,1,57,43,America/Los_Angeles
-cetauri@gmail.com,"ohsang.kwon",member,,"no email","allowed",2012,11,27,3,7,50,America/Los_Angeles
-charlieandrews.cwa@gmail.com,"Charlie Andrews",member,,"no email","allowed",2013,5,20,12,53,23,America/Los_Angeles
-charsyam@gmail.com,"Dae-Myung Kang",member,,"no email","allowed",2012,11,29,8,59,21,America/Los_Angeles
-choipd.net@gmail.com,"Myungjin Choi",member,,"no email","allowed",2012,11,30,18,53,44,America/Los_Angeles
-chopade.ritesh23@gmail.com,"Ritesh",member,,"no email","allowed",2013,1,7,1,50,43,America/Los_Angeles
-clee@escalation-point.com,"Cory Lee",member,,"no email","allowed",2013,4,25,13,30,18,America/Los_Angeles
-com.y.naka@gmail.com,"Yusuke NAKA",member,,"no email","allowed",2013,4,2,18,11,36,America/Los_Angeles
-creative.meidy@gmail.com,"meidydoang",member,,"no email","allowed",2012,7,29,21,2,33,America/Los_Angeles
-curioe@gmail.com,"curioe",member,,"no email","allowed",2012,7,25,0,27,26,America/Los_Angeles
-cywjackson@gmail.com,"Jackson Chung",member,,"no email","allowed",2013,10,15,14,54,48,America/Los_Angeles
-dan@code4.com.au,"Dan Miles",member,,"no email","allowed",2012,3,10,20,18,49,America/Los_Angeles
-daniel.simons1@gmail.com,"Daniel Simons",member,,"no email","allowed",2013,7,11,18,38,45,America/Los_Angeles
-dardo82@gmail.com,"Michele Venturi",member,,"no email","allowed",2013,4,26,5,5,55,America/Los_Angeles
-dave.aiello@after6services.com,,member,,"no email","allowed",2013,4,25,15,5,57,America/Los_Angeles
-david.barkman13@gmail.com,"David Barkman",member,,"no email","allowed",2013,1,12,8,55,9,America/Los_Angeles
-davidstoker@gmail.com,"David Stoker",member,,"no email","allowed",2012,9,10,7,1,18,America/Los_Angeles
-dereknkata@gmail.com,"Nkata Derrick",member,,"no email","allowed",2013,6,25,0,46,50,America/Los_Angeles
-deric.crago@gmail.com,"Deric Crago",member,,"no email","allowed",2012,7,12,21,41,35,America/Los_Angeles
-dev.scsu@gmail.com,,member,,"no email","allowed",2013,2,10,9,11,57,America/Los_Angeles
-dfzuluaga@gmail.com,"Diego Zuluaga",member,,"no email","allowed",2013,10,16,13,30,8,America/Los_Angeles
-dhrim00@gmail.com,"dohyoung rim",member,,"no email","allowed",2012,10,18,17,21,28,America/Los_Angeles
-dia.ab88@gmail.com,"Dia",member,,"no email","allowed",2013,2,6,8,29,33,America/Los_Angeles
-dipanjan122@gmail.com,"Deepanjan Majumdar",member,,"no email","allowed",2012,9,14,10,32,44,America/Los_Angeles
-dlewis@sympletech.com,"Daniel Lewis",member,,"no email","allowed",2012,9,14,12,6,49,America/Los_Angeles
-dmjohnson@apigee.com,"Dave Johnson",member,,"email","allowed",2013,9,26,11,34,35,America/Los_Angeles
-dnielsen@gmail.com,"Dave Nielsen",member,,"no email","allowed",2012,11,27,9,19,52,America/Los_Angeles
-dpchiesa@hotmail.com,"Dino Chiesa",member,,"no email","allowed",2012,9,15,16,6,17,America/Los_Angeles
-dulithaz@gmail.com,"Chan",member,,"no email","allowed",2013,9,22,19,39,27,America/Los_Angeles
-e.gunay@gmail.com,"Erdem Gunay",member,,"abridged","allowed",2013,6,23,23,19,20,America/Los_Angeles
-eanuff@apigee.com,"Ed Anuff",owner,,"no email","allowed",2012,8,9,14,53,6,America/Los_Angeles
-earthshipsolutions@gmail.com,"David Davida",member,,"no email","allowed",2013,6,16,10,5,0,America/Los_Angeles
-ed@anuff.com,"Ed Anuff",owner,,"email","allowed",2011,10,2,22,34,11,America/Los_Angeles
-ed@usergrid.com,"Ed Anuff",owner,,"no email","allowed",2011,9,27,14,31,49,America/Los_Angeles
-engin.deveci@gmail.com,"Engin Deveci",member,,"email","allowed",2012,9,4,3,17,59,America/Los_Angeles
-eort@apigee.com,"Ed Ort",member,,"no email","allowed",2012,9,11,11,43,18,America/Los_Angeles
-erhanabay@gmail.com,"Erhan Abay",member,,"abridged","allowed",2013,3,4,23,55,25,America/Los_Angeles
-eric.hanley@gmail.com,"eric",member,,"email","allowed",2012,6,25,10,38,22,America/Los_Angeles
-esecilakbas@gmail.com,"Ece Seçil AKBAŞ",member,,"abridged","allowed",2013,2,28,13,2,22,America/Los_Angeles
-esentuna1@gmail.com,"Esen Tuna",member,,"email","allowed",2013,2,25,23,18,5,America/Los_Angeles
-evyatar@doat.com,"Evyatar",member,,"no email","allowed",2012,7,23,23,1,48,America/Los_Angeles
-evyatron@gmail.com,,member,,"no email","allowed",2012,12,15,14,1,8,America/Los_Angeles
-feedback@apigee.com,"Apigee Feedback",member,,"no email","allowed",2013,5,28,15,3,7,America/Los_Angeles
-fernando@zenksworld.com,"Fernando Tomé",member,,"no email","allowed",2013,4,16,8,40,17,America/Los_Angeles
-feroz.panwaskar@gmail.com,"Feroz Panwaskar",member,,"no email","allowed",2013,5,15,9,0,0,America/Los_Angeles
-flozano@gmail.com,"Francisco Alejandro Lozano López",member,,"no email","allowed",2011,11,15,17,56,47,America/Los_Angeles
-garifuna@gmail.com,"Jorge Garifuna",member,,"email","allowed",2012,7,28,1,44,31,America/Los_Angeles
-garlandk@gmail.com,"gar",member,,"no email","allowed",2012,3,24,15,53,2,America/Los_Angeles
-george@mobitroll.no,"george",member,,"no email","allowed",2012,12,10,8,37,24,America/Los_Angeles
-gertjvr@gmail.com,"Gert Jansen van Rensburg",member,,"no email","allowed",2012,6,28,22,24,5,America/Los_Angeles
-ggem@acm.org,"ggem",member,,"abridged","allowed",2013,3,23,14,44,13,America/Los_Angeles
-ghartstein@gmail.com,"hartsteins",member,bouncing,"no email","allowed",2012,11,9,10,0,29,America/Los_Angeles
-gilberts55@gmail.com,"Gilbert EKOUE",member,,"no email","allowed",2013,8,28,11,15,9,America/Los_Angeles
-google@dbsolutions.net,"Dwight Barbour",member,,"no email","allowed",2013,8,19,21,13,56,America/Los_Angeles
-gorla.patricia@gmail.com,"Patricia Gorla",member,,"no email","allowed",2013,10,1,12,40,33,America/Los_Angeles
-grewis@apigee.com,"Greg Rewis",member,,"no email","allowed",2013,10,8,17,32,1,America/Los_Angeles
-gtrmjz@gmail.com,"Robert James",member,,"no email","allowed",2013,1,30,13,1,15,America/Los_Angeles
-guruguru@gmail.com,"Chris Kwon",member,,"no email","allowed",2012,8,8,18,13,30,America/Los_Angeles
-hal.newton@gmail.com,"Hal Newton",member,,"no email","allowed",2012,7,3,15,59,49,America/Los_Angeles
-hani.elabed@gmail.com,"hani",member,,"abridged","allowed",2011,10,3,8,44,47,America/Los_Angeles
-harigharan@softsuave.com,"Harigharan E",member,,"no email","allowed",2013,8,12,8,30,32,America/Los_Angeles
-harrybrook0@gmail.com,"Harry Smith",member,,"no email","allowed",2013,9,29,6,23,35,America/Los_Angeles
-hayesmp@thefrontside.net,"Michael Hayes",member,,"email","allowed",2013,10,21,14,42,35,America/Los_Angeles
-hellraiser101@gmail.com,"narendra chitrakar",member,,"no email","allowed",2012,10,14,20,52,6,America/Los_Angeles
-help@apigee.com,"Apigee Help",member,,"email","allowed",2013,5,28,15,6,21,America/Los_Angeles
-himanshuanandani@gmail.com,"himanshu anandani",member,,"no email","allowed",2013,8,29,1,52,23,America/Los_Angeles
-hjrascoe@gmail.com,"Justin Rascoe",member,,"no email","allowed",2013,5,28,19,55,1,America/Los_Angeles
-hmclaug@gmail.com,"Hunter Cash",member,,"no email","allowed",2013,3,2,13,56,37,America/Los_Angeles
-hotbabay22@yahoo.fr,"hotbabay22@yahoo.fr",member,bouncing,"no email","allowed",2012,2,20,22,57,31,America/Los_Angeles
-hvandenbulk@gmail.com,"Henri",member,,"no email","allowed",2012,11,7,21,26,17,America/Los_Angeles
-hwhelan@apigee.com,"Helen Whelan",member,,"no email","allowed",2012,7,11,11,49,42,America/Los_Angeles
-i.judekim@gmail.com,"Jude Kim",member,,"abridged","allowed",2012,8,23,5,5,35,America/Los_Angeles
-ian.cumberland@gmail.com,"fukurou",member,,"no email","allowed",2013,2,26,3,15,25,America/Los_Angeles
-ibrahim.abuelruzz@gmail.com,"Ibra Ibra",member,,"no email","allowed",2012,6,18,10,57,9,America/Los_Angeles
-ilya@eonmediagroup.com,"Ilya EONmediagroup",member,,"no email","allowed",2013,3,23,14,41,9,America/Los_Angeles
-indalyadav042@gmail.com,"INDAL YADAV",member,,"no email","not allowed",2013,10,16,23,58,19,America/Los_Angeles
-informationchef@gmail.com,"Information Chef",member,,"email","allowed",2011,10,3,10,59,41,America/Los_Angeles
-intuedge@gmail.com,"Doug May",member,,"no email","allowed",2013,6,26,22,13,59,America/Los_Angeles
-jameztcc@gmail.com,"James Tan",member,,"no email","allowed",2013,5,15,0,8,8,America/Los_Angeles
-jamison.k.bell@gmail.com,"Jamison K. Bell",member,,"no email","allowed",2012,6,22,3,50,19,America/Los_Angeles
-jarbunk@gmail.com,"Justin Mahar",member,,"no email","allowed",2012,9,7,10,57,38,America/Los_Angeles
-jason.duley@gmail.com,"Jason Duley",member,,"no email","allowed",2013,4,29,16,25,41,America/Los_Angeles
-jasonograves@gmail.com,"Jason Graves",member,,"no email","allowed",2013,6,30,19,45,18,America/Los_Angeles
-jay.ockers@gmail.com,"jockers",member,,"no email","allowed",2013,3,23,14,43,33,America/Los_Angeles
-JChung@apigee.com,"Jackson Chung",member,,"abridged","allowed",2012,7,24,17,54,6,America/Los_Angeles
-jdubray@gmail.com,"JJ",member,,"no email","allowed",2011,10,13,12,36,43,America/Los_Angeles
-jeffhsu.life@gmail.com,"J",member,,"no email","allowed",2013,6,4,23,36,44,America/Los_Angeles
-jeffrey.mock@gmail.com,"Jeffrey Mock",member,,"no email","allowed",2013,2,4,22,20,58,America/Los_Angeles
-jeremy@ignite.ms,"Jeremy Anticouni",member,,"no email","allowed",2013,4,19,18,44,43,America/Los_Angeles
-jerryhamby@gmail.com,"jerry hamby",member,,"digest","allowed",2012,8,8,11,31,45,America/Los_Angeles
-jinan62@gmail.com,"Jinan Toh",member,,"no email","allowed",2012,5,9,7,52,0,America/Los_Angeles
-jodyfranklin@gmail.com,"Jody Franklin",member,,"email","allowed",2012,9,2,17,16,31,America/Los_Angeles
-joel.yen@gmail.com,"Joel Yen",member,,"no email","allowed",2013,5,12,18,44,54,America/Los_Angeles
-johan.chouquet@gmail.com,"Johan Chouquet",member,,"no email","allowed",2013,6,3,21,4,31,America/Los_Angeles
-john.j.baumbach@gmail.com,"John B",member,,"no email","allowed",2013,2,10,15,13,55,America/Los_Angeles
-jose.vibar@gmail.com,"Jose Vibar",member,,"no email","allowed",2012,12,2,1,5,33,America/Los_Angeles
-jrodriguez@zenksworld.com,"Jorge Rodriguez",member,,"email","allowed",2013,5,13,4,15,56,America/Los_Angeles
-juanpablof@gmail.com,"J P",member,,"no email","allowed",2011,10,11,13,45,48,America/Los_Angeles
-julielieyal@gmail.com,"Julie Li",member,,"no email","allowed",2013,6,19,23,29,21,America/Los_Angeles
-jyotiman.prusty@gmail.com,"jyotiman",member,,"no email","allowed",2012,9,3,6,29,59,America/Los_Angeles
-kanitkar.abhishek@gmail.com,"Abhishek Kanitkar",member,,"no email","allowed",2012,10,16,0,25,17,America/Los_Angeles
-karim.varela@gmail.com,"Karim Varela",member,,"no email","allowed",2012,11,3,23,21,39,America/Los_Angeles
-kaytami@yahoo.com,"KC",member,,"no email","allowed",2012,5,31,0,10,13,America/Los_Angeles
-kevinecor@gmail.com,"mayorbyrne",member,,"no email","allowed",2012,11,22,12,5,34,America/Los_Angeles
-keyurchitnis@gmail.com,"keyur chitnis",member,,"no email","allowed",2012,11,7,13,32,31,America/Los_Angeles
-kglowacki@gmail.com,"Kris Glowacki",member,,"abridged","allowed",2013,5,20,11,19,32,America/Los_Angeles
-kirandeo87@gmail.com,"Kiran Deorukhkar",member,,"no email","allowed",2013,10,7,10,40,47,America/Los_Angeles
-kroll.rich@gmail.com,"Rich Kroll",member,,"abridged","allowed",2013,6,4,14,39,43,America/Los_Angeles
-kruti.barot.27@gmail.com,"Kruti Barot",member,,"no email","allowed",2013,9,20,23,21,27,America/Los_Angeles
-kswiber@gmail.com,"Kevin Swiber",member,,"email","allowed",2012,7,11,10,44,6,America/Los_Angeles
-ktopits@gmail.com,"Kirk Topits",member,,"no email","allowed",2013,5,28,10,8,1,America/Los_Angeles
-kurt.kanaskie@gmail.com,"Kurt Kanaskie",member,,"no email","allowed",2012,7,17,6,37,56,America/Los_Angeles
-laurent.pasquali@gmail.com,"Laurent PASQUALI",member,,"no email","allowed",2012,10,19,5,52,24,America/Los_Angeles
-lgo@apache.org,,member,,"email","allowed",2013,9,21,10,47,33,America/Los_Angeles
-lifesaverpc@gmail.com,"Tech LifesaverPC",member,,"digest","allowed",2012,8,22,15,9,10,America/Los_Angeles
-lloyd.reshard@gmail.com,"Lloyd Reshard",member,,"digest","allowed",2012,10,4,20,36,9,America/Los_Angeles
-luck0131.park@gmail.com,"geeluck",member,,"abridged","allowed",2012,7,27,8,5,7,America/Los_Angeles
-maciej.rakowicz@adleritech.com,"mr",member,,"no email","allowed",2013,8,9,0,26,8,America/Los_Angeles
-macs.dragon1@gmail.com,"macs",member,,"no email","allowed",2012,10,19,9,12,36,America/Los_Angeles
-marie.parnigoni@gmail.com,"Marie Parnigoni",member,,"no email","allowed",2012,4,30,1,32,26,America/Los_Angeles
-marius.andreiana@gmail.com,"Marius Andreiana",member,,"no email","allowed",2012,8,2,3,2,23,America/Los_Angeles
-mark@shust.com,"Mark Shust",member,,"no email","allowed",2012,6,27,21,58,18,America/Los_Angeles
-markmwaura3@gmail.com,"Mark Mwaura",member,,"abridged","allowed",2012,9,19,21,2,53,America/Los_Angeles
-marsh.gardiner@gmail.com,"Marsh Gardiner",member,,"no email","allowed",2012,8,29,9,52,43,America/Los_Angeles
-marsh@apigee.com,"Marsh Gardiner",member,,"digest","allowed",2012,3,28,22,3,14,America/Los_Angeles
-maruthi@apigee.com,"Maruti",member,,"email","allowed",2012,6,28,4,15,27,America/Los_Angeles
-mastung@gmail.com,"Hao TANG",member,,"no email","allowed",2013,8,7,0,34,58,America/Los_Angeles
-matt.mcclean@gmail.com,"Matt McClean",member,,"no email","allowed",2012,6,19,6,38,27,America/Los_Angeles
-matthias.gliwka@gmail.com,"Matthias Gliwka",member,,"no email","allowed",2013,7,19,11,46,55,America/Los_Angeles
-mattsobieray@gmail.com,"Matt Sobieray",member,,"no email","allowed",2013,3,23,14,43,44,America/Los_Angeles
-mattyg2k6@gmail.com,"Matthew Garcia",member,,"no email","allowed",2013,9,28,1,45,39,America/Los_Angeles
-mdobson@apigee.com,"Matthew Dobson",member,,"abridged","allowed",2013,3,12,10,52,14,America/Los_Angeles
-meinert@gmail.com,"Andrew Meinert",member,,"no email","allowed",2013,1,30,16,2,58,America/Los_Angeles
-metilluyoc@gmail.com,"Javier Herrera Núñez",member,,"digest","allowed",2013,10,13,21,45,0,America/Los_Angeles
-mhaller@escalation-point.com,,member,,"no email","allowed",2013,7,8,15,9,13,America/Los_Angeles
-michaelshen7@gmail.com,"Michael Shen",member,,"no email","allowed",2013,10,1,20,58,36,America/Los_Angeles
-michartmann@gmail.com,"Michael Ryberg Hartmann",member,,"no email","allowed",2013,1,31,15,26,6,America/Los_Angeles
-mikerex89@gmail.com,"Mike Rex",member,,"no email","allowed",2012,11,26,7,1,8,America/Los_Angeles
-mingfai.ma@gmail.com,"Mingfai Ma",member,,"email","allowed",2013,7,4,8,46,0,America/Los_Angeles
-mmalloy@apigee.com,"Michael Malloy",member,,"no email","allowed",2013,10,2,10,58,30,America/Los_Angeles
-monspo2@gmail.com,"Sung Moon",member,,"no email","allowed",2012,7,13,11,41,55,America/Los_Angeles
-mpease@gmail.com,"Matthew Pease",member,,"no email","allowed",2012,6,18,7,3,10,America/Los_Angeles
-mustafak26@gmail.com,"مصطفى كمال",member,,"no email","allowed",2012,9,23,5,7,7,America/Los_Angeles
-nathaniel.simpson@gmail.com,"Nathan Simpson",member,,"no email","allowed",2012,12,20,13,9,17,America/Los_Angeles
-neal.schneier@gmail.com,"Neal Schneier",member,,"no email","allowed",2012,10,5,10,41,21,America/Los_Angeles
-neil@cazcade.com,"Neil Ellis",member,,"no email","allowed",2013,5,29,14,23,13,America/Los_Angeles
-niranjan.karu@gmail.com,"Niranjan Karunanandham",member,,"no email","allowed",2013,9,23,3,1,45,America/Los_Angeles
-nmccall@apigee.com,"zznate",manager,,"email","not allowed",2012,3,16,11,12,35,America/Los_Angeles
-nrosolski@gmail.com,"nrosolski",member,,"no email","allowed",2012,7,3,21,51,52,America/Los_Angeles
-nvadiee@apigee.com,"Nima Vadiee",member,,"abridged","allowed",2013,2,6,11,24,0,America/Los_Angeles
-opolyo01@gmail.com,"Oleg Polyakov",member,,"no email","allowed",2012,10,21,11,22,29,America/Los_Angeles
-orcaelectronic@aol.com,"Simon Haller",member,,"digest","allowed",2013,5,25,4,51,52,America/Los_Angeles
-oyediyildiz@gmail.com,"Orhan Yediyıldız",member,,"no email","allowed",2012,6,4,15,17,15,America/Los_Angeles
-pashabhai@gmail.com,"Prashant Purkar",member,,"no email","allowed",2012,7,24,0,20,48,America/Los_Angeles
-paul.wood.iii@gmail.com,"Paul Wood",member,,"no email","allowed",2013,1,12,22,7,5,America/Los_Angeles
-prabhat143@gmail.com,"Prabhat Jha",member,,"email","allowed",2013,10,22,15,14,31,America/Los_Angeles
-pratik13butani@gmail.com,"Pratik Butani",member,,"no email","allowed",2013,5,1,21,58,27,America/Los_Angeles
-priteshpatel15@gmail.com,"Pritesh Patel",member,,"email","allowed",2013,4,25,16,41,24,America/Los_Angeles
-prwfreeman@gmail.com,"Hereford App Dev",member,,"abridged","allowed",2012,9,1,8,48,45,America/Los_Angeles
-radiozeux@googlemail.com,"Sebastian Albert",member,,"no email","allowed",2013,5,4,10,16,57,America/Los_Angeles
-rahul0704@gmail.com,"Rahul",member,,"no email","allowed",2012,9,7,11,35,9,America/Los_Angeles
-rahulkrishnan007@gmail.com,"Rahul Krishnan",member,,"abridged","allowed",2012,9,21,10,47,2,America/Los_Angeles
-rajanish@apigee.com,"Rajanish Gundlupet Jagadish",member,,"abridged","allowed",2012,10,10,9,58,18,America/Los_Angeles
-rajeev@apigee.com,"Rajeev Ramani",member,,"no email","allowed",2013,4,26,18,45,47,America/Los_Angeles
-rakesh.s@apigee.com,,member,,"no email","allowed",2013,5,28,17,8,24,America/Los_Angeles
-rakeshvk4u@gmail.com,"Rakesh VK",member,,"no email","allowed",2013,9,11,9,54,20,America/Los_Angeles
-ramesh.v1210@gmail.com,"Ramesh V",member,,"no email","allowed",2012,7,31,2,18,21,America/Los_Angeles
-rampradeepk@gmail.com,"Rampradeep",member,,"email","allowed",2012,10,8,23,38,18,America/Los_Angeles
-rasmus@moyo.la,"Rasmus Hansson",member,,"no email","allowed",2012,8,1,14,2,38,America/Los_Angeles
-raykyri@gmail.com,"Raymond Zhong",member,,"no email","allowed",2012,6,7,15,5,4,America/Los_Angeles
-rbwright@gmail.com,"rbw",member,,"email","allowed",2012,7,30,15,7,31,America/Los_Angeles
-reggieesposa@gmail.com,"reggie",member,,"abridged","allowed",2013,9,7,8,47,43,America/Los_Angeles
-rino@techsquare.a-tono.net,"Rino Seminara",member,,"no email","allowed",2012,3,12,8,59,46,America/Los_Angeles
-rippela@apigee.com,"Renu Ippela",member,,"no email","allowed",2013,9,26,11,34,53,America/Los_Angeles
-rnsksoft@gmail.com,"Kai Cherry",member,,"abridged","allowed",2012,7,31,15,23,30,America/Los_Angeles
-robert.djurasaj@gmail.com,"Robert Djurasaj",member,,"no email","allowed",2013,3,22,14,26,20,America/Los_Angeles
-robslama@gmail.com,"speedy2222",member,,"digest","allowed",2012,9,7,6,29,2,America/Los_Angeles
-rod@apigee.com,"Rod Simpson",member,,"email","allowed",2012,5,31,11,54,34,America/Los_Angeles
-rohit@tingendab.com,"Rohit Rai",member,,"no email","allowed",2013,9,17,12,10,49,America/Los_Angeles
-rubentrancoso@gmail.com,"Ruben Trancoso",member,,"abridged","allowed",2013,5,28,13,58,24,America/Los_Angeles
-rupenp525@gmail.com,"Rupen P",member,,"abridged","allowed",2013,6,29,10,21,12,America/Los_Angeles
-rutgerius@gmail.com,"Rutger van Dijk",member,,"no email","allowed",2012,8,17,7,57,38,America/Los_Angeles
-ryakkala@apigee.com,"RAppSy",member,bouncing,"no email","not allowed",2013,3,30,16,35,6,America/Los_Angeles
-sachin@webmobi.com,"S Anand",member,,"email","allowed",2012,6,3,17,15,54,America/Los_Angeles
-sagarborse.borse@gmail.com,"Sagar Borse",member,,"no email","allowed",2013,9,23,8,20,51,America/Los_Angeles
-sanchitml@gmail.com,"Sanchit Mittal",member,,"abridged","allowed",2012,11,13,16,58,50,America/Los_Angeles
-sangramsingh@apigee.com,"Sangram Singh Kunwar",member,,"no email","allowed",2012,10,10,9,49,23,America/Los_Angeles
-sarahkungfu32@gmail.com,"Sara Santos",member,,"abridged","allowed",2013,6,7,14,38,40,America/Los_Angeles
-sarthak@apigee.com,"Sarthak Gangopadhyay",member,,"email","allowed",2012,5,6,18,38,32,America/Los_Angeles
-sathishvj@gmail.com,"Sathish VJ",member,,"no email","allowed",2012,12,11,1,23,26,America/Los_Angeles
-sayem@asteriskbd.com,"Abu Sayem",member,,"no email","allowed",2013,8,2,3,9,44,America/Los_Angeles
-scott@apigee.com,"Scott Regan",member,,"email","allowed",2012,3,13,16,23,20,America/Los_Angeles
-seh40@students.uwf.edu,"Bobisback",member,,"no email","allowed",2012,10,9,15,52,12,America/Los_Angeles
-semplake@gmail.com,"semplake semplake",member,,"no email","allowed",2013,6,13,22,29,40,America/Los_Angeles
-SGanyo@apigee.com,"Scott Ganyo",member,,"email","allowed",2012,8,6,14,21,10,America/Los_Angeles
-shaozhuang.liu@gmail.com,"Shaozhuang Liu",member,,"digest","allowed",2013,6,3,13,25,10,America/Los_Angeles
-shivraj.cse@gmail.com,"Shivraj",member,,"no email","allowed",2012,9,5,2,10,23,America/Los_Angeles
-sinan.inel@gmail.com,"sinel",member,,"abridged","allowed",2012,7,28,8,2,21,America/Los_Angeles
-slknarayanan@gmail.com,"Narayanan S",member,,"email","allowed",2012,4,27,7,20,22,America/Los_Angeles
-snoopdave@gmail.com,"Dave Johnson",member,,"no email","allowed",2013,8,30,5,22,43,America/Los_Angeles
-sourajit.basac@gmail.com,"Sourajit Basak",member,,"no email","allowed",2012,7,22,22,53,59,America/Los_Angeles
-srujanmathur@gmail.com,"mathur",member,,"no email","allowed",2013,4,7,22,36,20,America/Los_Angeles
-starinsights@gmail.com,"JustinM",member,,"no email","allowed",2012,4,11,13,24,39,America/Los_Angeles
-stevem@currentww.com,"StMcPherson",member,,"email","allowed",2012,7,24,7,39,24,America/Los_Angeles
-stewmorg@gmail.com,"Stewart Morgan",member,,"no email","allowed",2013,6,20,7,29,16,America/Los_Angeles
-stliu@hibernate.org,"Shaozhuang Liu",member,,"no email","allowed",2013,5,29,20,39,33,America/Los_Angeles
-stovak@apigee.com,"STOVAK",member,,"no email","allowed",2012,10,3,12,15,6,America/Los_Angeles
-straut@apigee.com,"Steve Traut",member,,"abridged","allowed",2013,7,8,10,24,5,America/Los_Angeles
-sungju@softwaregeeks.org,,member,,"email","allowed",2013,5,2,18,35,0,America/Los_Angeles
-sunil.rnsit@gmail.com,"Sunil Kumar BT",member,,"no email","allowed",2012,11,7,10,41,40,America/Los_Angeles
-tahatayyab@gmail.com,"Taha Tayyab",member,,"no email","allowed",2013,6,1,23,31,51,America/Los_Angeles
-tebica@gmail.com,"Minwoo Park",member,,"no email","allowed",2012,7,24,23,45,45,America/Los_Angeles
-thani_rbd@hotmail.com,"Thaniara manchini teodoro",member,,"no email","allowed",2013,4,12,21,51,49,America/Los_Angeles
-thatboy.phantom@gmail.com,"Fab",member,,"no email","allowed",2013,1,31,16,43,4,America/Los_Angeles
-thinklog@gmail.com,"Joseph",member,,"email","allowed",2011,10,19,9,5,40,America/Los_Angeles
-thomas.holiyop@gmail.com,,member,,"no email","allowed",2013,7,4,2,59,41,America/Los_Angeles
-tim.anglade@gmail.com,"Tim Anglade",member,,"email","allowed",2013,10,10,19,59,52,America/Los_Angeles
-tim@apigee.com,"Tim Anglade",manager,,"no email","allowed - override",2012,4,3,9,52,47,America/Los_Angeles
-tim@radtastical.com,,member,,"abridged","allowed",2013,1,25,16,57,16,America/Los_Angeles
-tnine@apigee.com,"Todd Nine",manager,,"no email","allowed",2012,3,16,11,9,36,America/Los_Angeles
-todd.nine@gmail.com,"Todd Nine",manager,,"no email","allowed",2012,6,13,16,46,46,America/Los_Angeles
-tom.marrs@gmail.com,"Tom Marrs",member,,"no email","allowed",2013,3,22,14,26,26,America/Los_Angeles
-tomt555@gmail.com,"Tom",member,,"abridged","allowed",2012,8,12,19,52,39,America/Los_Angeles
-toto.thkim@gmail.com,"Taeho@KTH",member,,"no email","allowed",2012,11,7,0,0,26,America/Los_Angeles
-travis@cloudmetal.com,"Travis James",member,,"no email","allowed",2013,5,24,13,33,42,America/Los_Angeles
-traviskds@gmail.com,"Travis De Silva",member,,"no email","allowed",2013,1,11,5,12,29,America/Los_Angeles
-tsourapas@gmail.com,"Kiriakos",member,,"no email","allowed",2012,9,15,0,36,23,America/Los_Angeles
-tushar.m.s@gmail.com,"Tushar",member,,"no email","allowed",2012,6,18,21,0,49,America/Los_Angeles
-tutonehcc@gmail.com,"LogHomeFinishing Colorado",member,,"no email","allowed",2013,1,1,9,54,18,America/Los_Angeles
-twinhope108@gmail.com,"Eutiquio Chapa",member,,"no email","allowed",2012,9,23,23,27,51,America/Los_Angeles
-twong@twong.com,"Terry Wong",member,,"no email","allowed",2013,5,9,2,32,28,America/Los_Angeles
-v@vinspee.me,"Vince Speelman",member,,"no email","allowed",2013,7,1,17,52,25,America/Los_Angeles
-vaibhavb@gmail.com,"Vaibhav Bhandari",member,,"no email","allowed",2013,4,25,15,2,14,America/Los_Angeles
-vmasina@gmail.com,"Vijay Masina",member,,"no email","allowed",2013,2,26,16,27,17,America/Los_Angeles
-wattersjames@gmail.com,"James Watters",member,,"digest","allowed",2012,9,3,17,32,20,America/Los_Angeles
-wil.moore@wilmoore.com,"Wil Moore",member,,"no email","allowed",2013,3,23,14,40,42,America/Los_Angeles
-xybrek@gmail.com,"Xybrek",member,,"digest","allowed",2013,2,20,23,2,42,America/Los_Angeles
-yahya@g.ucla.edu,"Yahya Shaikh",member,,"no email","allowed",2013,6,15,21,33,6,America/Los_Angeles
-yamanoor@gmail.com,"Srihari Yamanoor",member,,"email","allowed",2012,10,9,16,54,2,America/Los_Angeles
-yasuzo100@gmail.com,"Yasuhide Kato",member,,"email","allowed",2013,9,27,4,44,40,America/Los_Angeles
-ymolists@gmail.com,"ymo",member,,"no email","allowed",2012,9,22,11,8,4,America/Los_Angeles
-yramiyer@gmail.com,"leo",member,,"abridged","allowed",2013,5,4,21,38,51,America/Los_Angeles
-ys1382@gmail.com,"Yusuf X",member,,"no email","allowed",2012,4,6,4,3,29,America/Los_Angeles
-zznate.m@gmail.com,"Nate McCall",member,,"no email","allowed",2013,10,9,7,40,38,America/Los_Angeles
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7cf253e8/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index f2026e6..79fe2bb 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -21,14 +21,14 @@ rat {
     // git
     '.git/**',
     '**/.gitignore',
-    
+
     // gradle
     '**/.gradle/**',
     'gradlew',
     'gradlew.bat',
     'gradle/wrapper/gradle-wrapper.properties',
     '**/build/**',
-    
+
     // IDE
     'etc/eclipseFormatterProfile.xml',
     'etc/intellijIdeaCodeStyle.xml',
@@ -47,14 +47,14 @@ rat {
     '**/*.json',
     '**/*.tx0',
     '**/*.txo',
-    
+
     // binary files
     '**/*.cer',
     '**/*.gfs',
     '**/keystore',
     '**/*.ser',
     '**/*.xls',
-    
+
     // other text files
     'gemfire-spark-connector/project/plugins.sbt',
     'gemfire-spark-connector/project/build.properties',
@@ -64,15 +64,31 @@ rat {
     'gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.xml',
     'gemfire-core/src/test/resources/com/gemstone/gemfire/management/internal/configuration/utils/*.xml',
 
-    // TODO - go through all the gemfire-site files!!
-    '**/gemfire-site/**',
- 
+    // gemfire-site: Adding exceptions to MIT/BSD licensed files
+    'gemfire-site/website/content/js/head.js',
+    'gemfire-site/website/content/js/html5shiv.js',
+    'gemfire-site/website/content/js/jquery-1.10.1.min.js',
+    'gemfire-site/website/content/js/jquery.icheck.js',
+    'gemfire-site/website/content/js/respond.min.js',
+    'gemfire-site/website/content/js/bootstrap.min.js',
+    'gemfire-site/website/content/css/bootflat-extensions.css',
+    'gemfire-site/website/content/css/bootflat-square.css',
+    'gemfire-site/website/content/css/bootflat.css',
+    'gemfire-site/website/content/bootstrap/bootstrap.min.css',
+    'gemfire-site/website/content/css/font-awesome.min.css',
+    'gemfire-site/website/lib/pandoc.template',
+    'gemfire-site/website/content/font/**',
+    // compiled logs and locks
+    'gemfire-site/website/tmp/**',
+    'gemfire-site/website/layouts/**',
+
+
     // ANTLR generated files
     'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/OQLLexer.java',
     'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/OQLLexerTokenTypes.java',
     'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/OQLParser.java',
     'gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/parse/OQLLexerTokenTypes.txt',
-    
+
     // Service META-INF
     '**/META-INF/services/org.xml.sax.ext.EntityResolver2',
     '**/META-INF/services/com.gemstone.gemfire.internal.cache.CacheService',
@@ -80,8 +96,8 @@ rat {
     '**/META-INF/services/org.springframework.shell.core.CommandMarker',
 
     // --- Other Licenses ---
-    
-    // Copied from other ASF projects 
+
+    // Copied from other ASF projects
     'gemfire-core/src/main/resources/com/gemstone/gemfire/admin/jmx/internal/doc-files/mbeans-descriptors.dtd',
     'gemfire-core/src/main/resources/com/gemstone/gemfire/admin/jmx/mbeans-descriptors.xml',
 
@@ -134,4 +150,3 @@ rat {
     'gemfire-web-api/src/main/webapp/docs/swagger-ui.min.js'
   ]
 }
-


[48/50] [abbrv] incubator-geode git commit: GEODE-608: Update rat excludes to ignore binary files

Posted by je...@apache.org.
GEODE-608: Update rat excludes to ignore binary files

Add png, jpg, gif, dia, pdf, keystore, truststore, and ico
to the excludes list.


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

Branch: refs/heads/feature/GEODE-14
Commit: 0851f186074be4265d8088e3a5e1969c85eed635
Parents: b826b4b
Author: Anthony Baker <ab...@pivotal.io>
Authored: Fri Dec 18 11:59:54 2015 -0800
Committer: Anthony Baker <ab...@pivotal.io>
Committed: Fri Dec 18 12:07:14 2015 -0800

----------------------------------------------------------------------
 gradle/rat.gradle | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0851f186/gradle/rat.gradle
----------------------------------------------------------------------
diff --git a/gradle/rat.gradle b/gradle/rat.gradle
index 63eff0b..2068b88 100644
--- a/gradle/rat.gradle
+++ b/gradle/rat.gradle
@@ -54,9 +54,16 @@ rat {
 
     // binary files
     '**/*.cer',
+    '**/*.dia',
     '**/*.gfs',
-    '**/keystore',
+    '**/*.gif',
+    '**/*.ico',
+    '**/*.jpg',
+    '**/*.keystore',
+    '**/*.pdf',
+    '**/*.png',
     '**/*.ser',
+    '**/*.truststore',
     '**/*.xls',
 
     // other text files


[44/50] [abbrv] incubator-geode git commit: GEODE-678: dlock reenter will no longer timeout

Posted by je...@apache.org.
GEODE-678: dlock reenter will no longer timeout


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

Branch: refs/heads/feature/GEODE-14
Commit: d31a85ff7898732857d5fd1abee8030c8beeca66
Parents: 0e366b0
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Dec 15 15:36:24 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 17 14:18:23 2015 -0800

----------------------------------------------------------------------
 .../internal/locks/DLockGrantor.java            |  5 +-
 .../locks/DLockReentrantLockJUnitTest.java      | 84 ++++++++++++++++++++
 2 files changed, 85 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d31a85ff/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
index b4e646c..a5b8e0f 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/locks/DLockGrantor.java
@@ -984,10 +984,7 @@ public class DLockGrantor {
     }
     try {
       checkDestroyed();
-      if (request.checkForTimeout()) {
-        // no cleanup here because we bypassed lock permissions
-        return 0;
-      }
+      // to fix GEODE-678 no longer call request.checkForTimeout
       DLockGrantToken grant = getGrantToken(request.getObjectName());
       if (grant == null) {
         if (logger.isTraceEnabled(LogMarker.DLS)) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d31a85ff/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java
new file mode 100644
index 0000000..2dbd231
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/internal/locks/DLockReentrantLockJUnitTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed.internal.locks;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+
+@Category(IntegrationTest.class)
+public class DLockReentrantLockJUnitTest {
+  final long id = 1213L;
+
+  private Cache cache;
+  private Region<Long, String> region;
+
+  @Before
+  public void setup() {
+    cache = new CacheFactory().set("mcast-port", "0").set("locators", "").create();
+    final RegionFactory<Long, String> regionFactory = cache.createRegionFactory("REPLICATE");
+    regionFactory.setScope(Scope.GLOBAL);
+    region = regionFactory.create("ReentrantLockRegion");
+    region.put(id, new String("TestValue1"));
+  }
+
+  @After
+  public void tearDown() {
+    cache.close();
+  }
+
+  /**
+   * Tests GEM-96/GEODE-678
+   */
+  @Test
+  public void testReentrantLock() throws Exception {
+
+    Assert.assertEquals(Scope.GLOBAL, region.getAttributes().getScope());
+
+    final Lock lock1 = region.getDistributedLock(id);
+    final Lock lock2 = region.getDistributedLock(id);
+
+    for (int i = 0; i < 50; i++) {
+      lock1.lock();
+      boolean reenteredLock = false;
+      try {
+        reenteredLock = lock2.tryLock(1, TimeUnit.NANOSECONDS);
+        if (!reenteredLock) {
+          System.out.println("ERROR: could not reenter lock");
+        }
+        Assert.assertTrue("Failed getting lock at 2:" + i, reenteredLock);
+      } finally {
+        if (reenteredLock) {
+          lock2.unlock();
+        }
+        lock1.unlock();
+      }
+    }
+  }
+}


[45/50] [abbrv] incubator-geode git commit: GEODE-680: refactor Chunk to cleanup unit tests 1) Promoted constant fields to default access level. 2) Modified getAddressForReading to delegate to getUnsafeAddress.

Posted by je...@apache.org.
GEODE-680: refactor Chunk to cleanup unit tests
1) Promoted constant fields to default access level.
2) Modified getAddressForReading to delegate to getUnsafeAddress.


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

Branch: refs/heads/feature/GEODE-14
Commit: bb794ee205400ac4e5655cc1e6410889d29f28b1
Parents: d31a85f
Author: Sai Boorlagadda <sb...@pivotal.io>
Authored: Wed Dec 16 13:03:19 2015 -0800
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Dec 17 14:25:44 2015 -0800

----------------------------------------------------------------------
 .../gemfire/internal/offheap/Chunk.java         | 26 ++++++++++----------
 .../internal/offheap/GemFireChunkJUnitTest.java | 23 ++++++++---------
 2 files changed, 23 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb794ee2/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
index ed4bc43..4d1252d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/Chunk.java
@@ -74,21 +74,21 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
      * a delta whose max value would be HUGE_MULTIPLE-1.
      * The lower two bytes contains the use count.
      */
-    private final static int REF_COUNT_OFFSET = 4;
+    final static int REF_COUNT_OFFSET = 4;
     /**
      * The upper two bits are used for the isSerialized
      * and isCompressed flags.
      */
-    private final static int IS_SERIALIZED_BIT =    0x80000000;
-    private final static int IS_COMPRESSED_BIT =    0x40000000;
-    private final static int SRC_TYPE_MASK = 0x38000000;
-    private final static int SRC_TYPE_SHIFT = 16/*refCount*/+8/*dataSize*/+3/*magicSize*/;
-    private final static int MAGIC_MASK = 0x07000000;
-    private final static int MAGIC_NUMBER = 0x05000000;
-    private final static int DATA_SIZE_DELTA_MASK = 0x00ff0000;
-    private final static int DATA_SIZE_SHIFT = 16;
-    private final static int REF_COUNT_MASK =       0x0000ffff;
-    private final static int MAX_REF_COUNT = 0xFFFF;
+    final static int IS_SERIALIZED_BIT =    0x80000000;
+    final static int IS_COMPRESSED_BIT =    0x40000000;
+    final static int SRC_TYPE_MASK = 0x38000000;
+    final static int SRC_TYPE_SHIFT = 16/*refCount*/+8/*dataSize*/+3/*magicSize*/;
+    final static int MAGIC_MASK = 0x07000000;
+    final static int MAGIC_NUMBER = 0x05000000;
+    final static int DATA_SIZE_DELTA_MASK = 0x00ff0000;
+    final static int DATA_SIZE_SHIFT = 16;
+    final static int REF_COUNT_MASK =       0x0000ffff;
+    final static int MAX_REF_COUNT = 0xFFFF;
     final static long FILL_PATTERN = 0x3c3c3c3c3c3c3c3cL;
     final static byte FILL_BYTE = 0x3c;
     
@@ -376,8 +376,8 @@ import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
     }
 
     public long getAddressForReading(int offset, int size) {
-      assert offset+size <= getDataSize();
-      return getBaseDataAddress() + offset;
+      //delegate to getUnsafeAddress - as both the methods does return the memory address from given offset
+      return getUnsafeAddress(offset, size);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bb794ee2/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
index 20cb8dc..81a59e3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/offheap/GemFireChunkJUnitTest.java
@@ -255,7 +255,7 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
 
-    int headerBeforeSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+    int headerBeforeSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
 
     assertThat(chunk.isSerialized()).isFalse();
 
@@ -263,9 +263,9 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     assertThat(chunk.isSerialized()).isTrue();
 
-    int headerAfterSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+    int headerAfterSerializedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
 
-    assertThat(headerAfterSerializedBitSet).isEqualTo(headerBeforeSerializedBitSet | 0x80000000/* IS_SERIALIZED_BIT */);
+    assertThat(headerAfterSerializedBitSet).isEqualTo(headerBeforeSerializedBitSet | Chunk.IS_SERIALIZED_BIT);
 
     chunk.release();
   }
@@ -289,7 +289,7 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     GemFireChunk chunk = (GemFireChunk) ma.allocateAndInitialize(regionEntryValueAsBytes, isSerialized, isCompressed, GemFireChunk.TYPE);
 
-    int headerBeforeCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+    int headerBeforeCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
 
     assertThat(chunk.isCompressed()).isFalse();
 
@@ -297,9 +297,9 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
 
     assertThat(chunk.isCompressed()).isTrue();
 
-    int headerAfterCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4/* REF_COUNT_OFFSET */);
+    int headerAfterCompressedBitSet = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
 
-    assertThat(headerAfterCompressedBitSet).isEqualTo(headerBeforeCompressedBitSet | 0x40000000/* IS_SERIALIZED_BIT */);
+    assertThat(headerAfterCompressedBitSet).isEqualTo(headerBeforeCompressedBitSet | Chunk.IS_COMPRESSED_BIT);
 
     chunk.release();
   }
@@ -730,11 +730,8 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
   public void retainShouldThrowExceptionAfterMaxNumberOfTimesRetained() {
     GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
 
-    // max retain Chunk.MAX_REF_COUNT
-    int MAX_REF_COUNT = 0xFFFF;
-
     // loop though and invoke retain for MAX_REF_COUNT-1 times, as create chunk above counted as one reference
-    for (int i = 0; i < MAX_REF_COUNT - 1; i++)
+    for (int i = 0; i < Chunk.MAX_REF_COUNT - 1; i++)
       chunk.retain();
 
     // invoke for the one more time should throw exception
@@ -891,7 +888,7 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
   public void getSrcTypeOrdinalFromAddressShouldReturnOrdinal() {
     GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
 
-    assertThat(Chunk.getSrcTypeOrdinal(chunk.getMemoryAddress())).isEqualTo(4);
+    assertThat(Chunk.getSrcTypeOrdinal(chunk.getMemoryAddress())).isEqualTo(Chunk.SRC_TYPE_GFE >> Chunk.SRC_TYPE_SHIFT);
 
     chunk.release();
   }
@@ -900,8 +897,8 @@ public class GemFireChunkJUnitTest extends AbstractStoredObjectTestBase {
   public void getSrcTypeOrdinalFromRawBitsShouldReturnOrdinal() {
     GemFireChunk chunk = createValueAsUnserializedStoredObject(getValue());
 
-    int rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + 4 /* REF_COUNT_OFFSET */);
-    assertThat(Chunk.getSrcTypeOrdinalFromRawBits(rawBits)).isEqualTo(4);
+    int rawBits = UnsafeMemoryChunk.readAbsoluteIntVolatile(chunk.getMemoryAddress() + Chunk.REF_COUNT_OFFSET);
+    assertThat(Chunk.getSrcTypeOrdinalFromRawBits(rawBits)).isEqualTo(Chunk.SRC_TYPE_GFE >> Chunk.SRC_TYPE_SHIFT);
 
     chunk.release();
   }


[10/50] [abbrv] incubator-geode git commit: GEODE-291: Prevent stderr noise from com.gemstone.gemfire.test.process

Posted by je...@apache.org.
GEODE-291: Prevent stderr noise from com.gemstone.gemfire.test.process

Change classes in com.gemstone.gemfire.test.process package:

* Remove unused methods and variables..

* Change waitFor() to match the Process.waitFor(long, TimeUnit) method in
JDK 1.8. Pass in timeout parameters from ProcessWrapper instead of
hardcoding it.

* Separate waitFor() into waitFor(long, TimeUnit) and start(). Previously
waitFor() was performing both of these actions.

* Improve debugging by: 1) adding minor lifecycle to ProcessOutputReader,
2) include command string in stack trace of ProcessStreamReader. This
enabled identification of the source of unwanted stderr output.

Change DistributedSystem to enable testing:

* Change DistributedSystem to enable better unit testing by allowing tests
to provide different locations for gemfire.properties with
gemfirePropertyFile. Make same changes for gfsecurity.properties and
gemfireSecurityPropertyFile.

* Create unit and integration tests for DistributedSystem class.

Change tests that use com.gemstone.gemfire.test.process package:

* Use DistributedSystem.PROPERTIES_FILE_PROPERTY to override location of
gemfire.properties.

* Separate integration tests from UnitTest files to IntegrationTests.

* Add and use Catch-Exception for better unit testing.

* Remove unused code and class.


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

Branch: refs/heads/feature/GEODE-14
Commit: fe295940c2c61a97c84ae333574e80001962ba15
Parents: 5b35e43
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Sep 15 16:15:33 2015 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Fri Dec 11 16:01:43 2015 -0800

----------------------------------------------------------------------
 build.gradle                                    |   4 +-
 .../gemfire/distributed/AbstractLauncher.java   |   2 +-
 .../gemfire/distributed/DistributedSystem.java  | 208 ++++++++++---
 .../AbstractLauncherIntegrationJUnitTest.java   |  71 +++++
 .../distributed/AbstractLauncherJUnitTest.java  |  39 +--
 .../distributed/CommonLauncherTestSuite.java    |  65 ----
 .../DistributedSystemIntegrationJUnitTest.java  |  91 ++++++
 .../distributed/DistributedSystemJUnitTest.java |  78 +++++
 .../LocatorLauncherIntegrationJUnitTest.java    | 248 +++++++++++++++
 .../distributed/LocatorLauncherJUnitTest.java   | 154 +--------
 .../ServerLauncherIntegrationJUnitTest.java     | 312 +++++++++++++++++++
 .../distributed/ServerLauncherJUnitTest.java    | 183 +----------
 .../test/process/ProcessOutputReader.java       | 101 +++---
 .../test/process/ProcessStreamReader.java       |  42 +--
 .../gemfire/test/process/ProcessWrapper.java    |  10 +-
 gradle/dependency-versions.properties           |   4 +-
 16 files changed, 1064 insertions(+), 548 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 9c8ac44..f464dc3 100755
--- a/build.gradle
+++ b/build.gradle
@@ -325,9 +325,11 @@ subprojects {
     compile 'org.springframework:spring-webmvc:' + project.'springframework.version'
     compile 'com.github.stephenc.findbugs:findbugs-annotations:' + project.'stephenc-findbugs.version'
 
-    testCompile 'com.jayway.awaitility:awaitility:' + project.'awaitility.version'
     testCompile 'com.github.stefanbirkner:system-rules:' + project.'system-rules.version'
+    testCompile 'com.jayway.awaitility:awaitility:' + project.'awaitility.version'
     testCompile 'edu.umd.cs.mtc:multithreadedtc:' + project.'multithreadedtc.version'
+    testCompile 'eu.codearte.catch-exception:catch-exception:' + project.'catch-exception.version'
+    testCompile 'eu.codearte.catch-exception:catch-throwable:' + project.'catch-throwable.version'
     testCompile 'junit:junit:' + project.'junit.version'
     testCompile 'org.assertj:assertj-core:' + project.'assertj-core.version'
     testCompile 'org.mockito:mockito-core:' + project.'mockito-core.version'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java
index bfd3de7..205f501 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/AbstractLauncher.java
@@ -170,7 +170,7 @@ public abstract class AbstractLauncher<T extends Comparable<T>> implements Runna
       catch (Exception e) {
         try {
           // not in the file system, try the classpath
-          properties.load(AbstractLauncher.class.getResourceAsStream(DistributedSystem.PROPERTY_FILE));
+          properties.load(AbstractLauncher.class.getResourceAsStream(DistributedSystem.getPropertiesFile()));
         }
         catch (Exception ignore) {
           // not in the file system or the classpath; gemfire.properties does not exist

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
index e2fccd9..159db86 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/DistributedSystem.java
@@ -1943,26 +1943,59 @@ public abstract class DistributedSystem implements StatisticsFactory {
    */
   public abstract String getName();
 
-//   /**
-//    * Fires an "informational" <code>SystemMembershipEvent</code> that
-//    * is delivered to all {@link
-//    * com.gemstone.gemfire.admin.SystemMembershipListener}s.
-//    *
-//    * @param callback
-//    *        A user-specified object that is delivered with the {@link
-//    *        com.gemstone.gemfire.admin.SystemMembershipEvent}
-//    *        triggered by invoking this method.
-//    *
-//    * @see com.gemstone.gemfire.admin.SystemMembershipListener#memberInfo
-//    *
-//    * @since 4.0
-//    */
-//   public abstract void fireInfoEvent(Object callback);
+  /**
+   * The <code>PROPERTIES_FILE_PROPERTY</code> is the system property
+   * that can be used to specify the name of the properties file that the 
+   * connect method will check for when it looks for a properties file. Unless
+   * the value specifies the fully qualified path to the file, the file will 
+   * be searched for, in order, in the following directories:
+   * <ol>
+   * <li> the current directory
+   * <li> the home directory
+   * <li> the class path
+   * </ol>
+   * Only the first file found will be used.
+   * <p>
+   * The default value is {@link #PROPERTIES_FILE_DEFAULT}. However 
+   * if the <code>PROPERTIES_FILE_PROPERTY</code> is set then its value 
+   * will be used instead of the default. If this value is a relative file
+   * system path then the above search is done.  If it is an absolute
+   * file system path then that file must exist; no search for it is
+   * done.
+   * 
+   * @see #PROPERTIES_FILE_DEFAULT
+   * @see #getPropertiesFile()
+   * @since 9.0
+   */
+  public static final String PROPERTIES_FILE_PROPERTY = "gemfirePropertyFile";
+  
+  /** 
+   * The default value of <code>PROPERTIES_FILE_PROPERTY</code> is
+   * <code>"gemfire.properties"</code>. The location of the file will be 
+   * resolved during connect as described for {@link #PROPERTIES_FILE_PROPERTY}.
+   * 
+   * @see #PROPERTIES_FILE_PROPERTY
+   * @see #getPropertiesFile()
+   * @since 9.0
+   */
+  public static final String PROPERTIES_FILE_DEFAULT = "gemfire.properties";
 
   /**
+   * Returns the current value of {@link #PROPERTIES_FILE_PROPERTY} system 
+   * property if set or the default value {@link #PROPERTIES_FILE_DEFAULT}.
+   * 
+   * @see #PROPERTIES_FILE_PROPERTY
+   * @see #PROPERTIES_FILE_DEFAULT
+   * @since 9.0
+   */
+  public static String getPropertiesFile() {
+	return System.getProperty(PROPERTIES_FILE_PROPERTY, PROPERTIES_FILE_DEFAULT);
+  }
+  
+  /**
    * The <code>PROPERTY_FILE</code> is the name of the
-   * property file that the connect method will check for when
-   * it looks for a property file.
+   * properties file that the connect method will check for when
+   * it looks for a properties file.
    * The file will be searched for, in order, in the following directories:
    * <ol>
    * <li> the current directory
@@ -1978,60 +2011,139 @@ public abstract class DistributedSystem implements StatisticsFactory {
    * system path then the above search is done.  If it is an absolute
    * file system path then that file must exist; no search for it is
    * done.
+   * 
+   * @see #getPropertiesFile()
    * @since 5.0
-   *  */
-  public static final String PROPERTY_FILE = System.getProperty("gemfirePropertyFile", "gemfire.properties");
+   * @deprecated As of 9.0, please use {@link #getPropertiesFile()} instead. 
+   */
+  public static String PROPERTY_FILE = getPropertiesFile();
+
+  /**
+   * The <code>SECURITY_PROPERTIES_FILE_PROPERTY</code> is the system property
+   * that can be used to specify the name of the property file that the 
+   * connect method will check for when it looks for a property file. Unless
+   * the value specifies the fully qualified path to the file, the file will 
+   * be searched for, in order, in the following directories:
+   * <ol>
+   * <li> the current directory
+   * <li> the home directory
+   * <li> the class path
+   * </ol>
+   * Only the first file found will be used.
+   * <p>
+   * The default value is {@link #SECURITY_PROPERTIES_FILE_DEFAULT}. However 
+   * if the <code>SECURITY_PROPERTIES_FILE_PROPERTY</code> is set then its value 
+   * will be used instead of the default. If this value is a relative file
+   * system path then the above search is done.  If it is an absolute
+   * file system path then that file must exist; no search for it is
+   * done.
+   * 
+   * @see #SECURITY_PROPERTIES_FILE_DEFAULT
+   * @see #getSecurityPropertiesFile()
+   * @since 9.0
+   */
+  public static final String SECURITY_PROPERTIES_FILE_PROPERTY = "gemfireSecurityPropertyFile";
+  
+  /** 
+   * The default value of <code>SECURITY_PROPERTIES_FILE_PROPERTY</code> is
+   * <code>"gfsecurity.properties"</code>. The location of the file will be 
+   * resolved during connect as described for {@link #SECURITY_PROPERTIES_FILE_PROPERTY}.
+   * 
+   * @see #SECURITY_PROPERTIES_FILE_PROPERTY
+   * @see #getSecurityPropertiesFile()
+   * @since 9.0
+   */
+  public static final String SECURITY_PROPERTIES_FILE_DEFAULT = "gfsecurity.properties";
 
   /**
+   * Returns the current value of {@link #SECURITY_PROPERTIES_FILE_PROPERTY} system 
+   * property if set or the default value {@link #SECURITY_PROPERTIES_FILE_DEFAULT}.
+   * 
+   * @see #SECURITY_PROPERTIES_FILE_PROPERTY
+   * @see #SECURITY_PROPERTIES_FILE_DEFAULT
+   * @since 9.0
+   */
+  public static String getSecurityPropertiesFile() {
+	return System.getProperty(SECURITY_PROPERTIES_FILE_PROPERTY, SECURITY_PROPERTIES_FILE_DEFAULT);
+  }
+  
+  /**
    * The <code>SECURITY_PROPERTY_FILE</code> is the name of the
-     * property file that the connect method will check for when
-     * it looks for a security property file.
-     * The file will be searched for, in order, in the following directories:
-     * <ol>
-     * <li> the current directory
-     * <li> the home directory
-     * <li> the class path
-     * </ol>
-     * Only the first file found will be used.
-     * <p>
-     * The default value of SECURITY_PROPERTY_FILE is
-     * <code>"gfsecurity.properties"</code>.  However if the
-     * "gemfireSecurityPropertyFile" system property is set then its value is
-     * the value of SECURITY_PROPERTY_FILE. If this value is a relative file
-     * system path then the above search is done.  If it is an absolute
-     * file system path then that file must exist; no search for it is
-     * done.
-     * @since 6.6.2
+   * property file that the connect method will check for when
+   * it looks for a security property file.
+   * The file will be searched for, in order, in the following directories:
+   * <ol>
+   * <li> the current directory
+   * <li> the home directory
+   * <li> the class path
+   * </ol>
+   * Only the first file found will be used.
+   * <p>
+   * The default value of SECURITY_PROPERTY_FILE is
+   * <code>"gfsecurity.properties"</code>.  However if the
+   * "gemfireSecurityPropertyFile" system property is set then its value is
+   * the value of SECURITY_PROPERTY_FILE. If this value is a relative file
+   * system path then the above search is done.  If it is an absolute
+   * file system path then that file must exist; no search for it is
+   * done.
+   * 
+   * @see #getSecurityPropertiesFile()
+   * @since 6.6.2
+   * @deprecated As of 9.0, please use {@link #getSecurityPropertiesFile()} instead. 
    */
-  public static final String SECURITY_PROPERTY_FILE = System.getProperty("gemfireSecurityPropertyFile",
-    "gfsecurity.properties");
+  public static String SECURITY_PROPERTY_FILE = getSecurityPropertiesFile();
+
+  /**
+   * Gets an <code>URL</code> for the properties file, if one can be found,
+   * that the connect method will use as its properties file.
+   * <p>
+   * See {@link #PROPERTIES_FILE_PROPERTY} for information on the name of
+   * the properties file and what locations it will be looked for in.
+   * 
+   * @return a <code>URL</code> that names the GemFire property file.
+   *    Null is returned if no property file was found.
+   * @see #PROPERTIES_FILE_PROPERTY
+   * @see #PROPERTIES_FILE_DEFAULT
+   * @see #getPropertiesFile()
+   * @since 9.0
+   */
+  public static URL getPropertiesFileURL() {
+    return getFileURL(getPropertiesFile());
+  }
 
   /**
    * Gets an <code>URL</code> for the property file, if one can be found,
    * that the connect method will use as its property file.
    * <p>
-   * See {@link #PROPERTY_FILE} for information on the name of
+   * See {@link #PROPERTIES_FILE_PROPERTY} for information on the name of
    * the property file and what locations it will be looked for in.
+   * 
    * @return a <code>URL</code> that names the GemFire property file.
    *    Null is returned if no property file was found.
+   * @see #getPropertiesFileURL()
    * @since 5.0
+   * @deprecated As of 9.0, please use {@link #getPropertiesFileURL()}
    */
   public static URL getPropertyFileURL() {
-    return getFileURL(PROPERTY_FILE);
+    return getPropertiesFileURL();
   }
 
   /**
-   * Gets an <code>URL</code> for the security property file, if one can be found,
-   * that the connect method will use as its property file.
+   * Gets an <code>URL</code> for the security properties file, if one can be found,
+   * that the connect method will use as its properties file.
    * <p>
-   * See {@link #SECURITY_PROPERTY_FILE} for information on the name of
-   * the property file and what locations it will be looked for in.
-   * @return a <code>URL</code> that names the GemFire security property file.
-   *    Null is returned if no property file was found.
+   * See {@link #SECURITY_PROPERTIES_FILE_PROPERTY} for information on the name of
+   * the properties file and what locations it will be looked for in.
+   * 
+   * @return a <code>URL</code> that names the GemFire security properties file.
+   *    Null is returned if no properties file was found.
+   * @see #SECURITY_PROPERTIES_FILE_PROPERTY
+   * @see #SECURITY_PROPERTIES_FILE_DEFAULT
+   * @see #getSecurityPropertiesFile()
    * @since 6.6.2
    */
   public static URL getSecurityPropertiesFileURL() {
-    return getFileURL(SECURITY_PROPERTY_FILE);
+    return getFileURL(getSecurityPropertiesFile());
   }
 
   private static URL getFileURL(String fileName) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationJUnitTest.java
new file mode 100755
index 0000000..745090d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherIntegrationJUnitTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import static org.assertj.core.api.Assertions.*;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests for AbstractLauncher class. These tests require file system I/O.
+ */
+@Category(IntegrationTest.class)
+public class AbstractLauncherIntegrationJUnitTest {
+
+  @Rule
+  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+  
+  @Rule
+  public final TestName testName = new TestName();
+  
+  private File gemfirePropertiesFile;
+  private Properties expectedGemfireProperties;
+  
+  @Before
+  public void setUp() throws Exception {
+    this.gemfirePropertiesFile = this.temporaryFolder.newFile("gemfire.properties");
+    
+    this.expectedGemfireProperties = new Properties();
+    this.expectedGemfireProperties.setProperty(DistributionConfig.NAME_NAME, "memberOne");
+    this.expectedGemfireProperties.setProperty(DistributionConfig.GROUPS_NAME, "groupOne, groupTwo");
+    this.expectedGemfireProperties.store(new FileWriter(this.gemfirePropertiesFile, false), this.testName.getMethodName());
+
+    assertThat(this.gemfirePropertiesFile).isNotNull();
+    assertThat(this.gemfirePropertiesFile.exists()).isTrue();
+    assertThat(this.gemfirePropertiesFile.isFile()).isTrue();
+  }
+  
+  @Test
+  public void testLoadGemFirePropertiesFromFile() throws Exception {
+    final Properties actualGemFireProperties = AbstractLauncher.loadGemFireProperties(this.gemfirePropertiesFile.toURI().toURL());
+
+    assertThat(actualGemFireProperties).isNotNull();
+    assertThat(actualGemFireProperties).isEqualTo(this.expectedGemfireProperties);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java
index 399c78f..7b8a6bb 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/AbstractLauncherJUnitTest.java
@@ -18,8 +18,6 @@ package com.gemstone.gemfire.distributed;
 
 import static org.junit.Assert.*;
 
-import java.io.File;
-import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.Properties;
@@ -45,10 +43,10 @@ import org.junit.experimental.categories.Category;
  * @since 7.0
  */
 @Category(UnitTest.class)
-public class AbstractLauncherJUnitTest extends CommonLauncherTestSuite {
+public class AbstractLauncherJUnitTest {
 
   protected AbstractLauncher<?> createAbstractLauncher(final String memberName, final String memberId) {
-    return new TestServiceLauncher(memberName, memberId);
+    return new FakeServiceLauncher(memberName, memberId);
   }
 
   @Test
@@ -111,26 +109,6 @@ public class AbstractLauncherJUnitTest extends CommonLauncherTestSuite {
   }
 
   @Test
-  public void testLoadGemFirePropertiesFromFile() throws IOException {
-    final Properties expectedGemfireProperties = new Properties();
-
-    expectedGemfireProperties.setProperty(DistributionConfig.NAME_NAME, "memberOne");
-    expectedGemfireProperties.setProperty(DistributionConfig.GROUPS_NAME, "groupOne, groupTwo");
-
-    final File gemfirePropertiesFile = writeGemFirePropertiesToFile(expectedGemfireProperties, "gemfire.properties",
-      "Test gemfire.properties file for AbstractLauncherJUnitTest.testLoadGemFirePropertiesFromFile");
-
-    assertNotNull(gemfirePropertiesFile);
-    assertTrue(gemfirePropertiesFile.isFile());
-
-    final Properties actualGemFireProperties = AbstractLauncher.loadGemFireProperties(
-      gemfirePropertiesFile.toURI().toURL());
-
-    assertNotNull(actualGemFireProperties);
-    assertEquals(expectedGemfireProperties, actualGemFireProperties);
-  }
-
-  @Test
   public void testGetDistributedSystemProperties() {
     AbstractLauncher<?> launcher = createAbstractLauncher("memberOne", "1");
 
@@ -274,12 +252,12 @@ public class AbstractLauncherJUnitTest extends CommonLauncherTestSuite {
       TimeUnit.DAYS.toMillis(2) + TimeUnit.HOURS.toMillis(1) + TimeUnit.MINUTES.toMillis(30) + TimeUnit.SECONDS.toMillis(1)));
   }
 
-  protected static final class TestServiceLauncher extends AbstractLauncher<String> {
+  protected static final class FakeServiceLauncher extends AbstractLauncher<String> {
 
     private final String memberId;
     private final String memberName;
 
-    public TestServiceLauncher(final String memberName, final String memberId) {
+    public FakeServiceLauncher(final String memberName, final String memberId) {
       this.memberId = memberId;
       this.memberName = memberName;
     }
@@ -289,6 +267,7 @@ public class AbstractLauncherJUnitTest extends CommonLauncherTestSuite {
       return false;
     }
 
+    @Override
     public String getLogFileName() {
       throw new UnsupportedOperationException("Not Implemented!");
     }
@@ -298,22 +277,22 @@ public class AbstractLauncherJUnitTest extends CommonLauncherTestSuite {
       return memberId;
     }
 
+    @Override
     public String getMemberName() {
       return memberName;
     }
 
+    @Override
     public Integer getPid() {
       throw new UnsupportedOperationException("Not Implemented!");
     }
 
+    @Override
     public String getServiceName() {
       return "TestService";
     }
 
-    public String getId() {
-      throw new UnsupportedOperationException("Not Implemented!");
-    }
-
+    @Override
     public void run() {
       throw new UnsupportedOperationException("Not Implemented!");
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/CommonLauncherTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/CommonLauncherTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/CommonLauncherTestSuite.java
deleted file mode 100644
index 94ba320..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/CommonLauncherTestSuite.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package com.gemstone.gemfire.distributed;
-
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.Properties;
-
-import org.junit.Rule;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.TestName;
-
-/**
- * The CommonLauncherTestSuite is a base class for encapsulating reusable functionality across the various, specific
- * launcher test suites.
- * </p>
- * @author John Blum
- * @see com.gemstone.gemfire.distributed.AbstractLauncherJUnitTest
- * @see com.gemstone.gemfire.distributed.LocatorLauncherJUnitTest
- * @see com.gemstone.gemfire.distributed.ServerLauncherJUnitTest
- * @since 7.0
- */
-public abstract class CommonLauncherTestSuite {
-
-  @Rule
-  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-  
-  @Rule
-  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
-  
-  @Rule
-  public final TestName testName = new TestName();
-  
-  protected File writeGemFirePropertiesToFile(final Properties gemfireProperties,
-                                              final String filename,
-                                              final String comment)
-  {
-    try {
-      final File gemfirePropertiesFile = this.temporaryFolder.newFile(filename);
-      gemfireProperties.store(new FileWriter(gemfirePropertiesFile, false), comment);
-      return gemfirePropertiesFile;
-    }
-    catch (IOException e) {
-      return null;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemIntegrationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemIntegrationJUnitTest.java
new file mode 100755
index 0000000..d54dfd0
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemIntegrationJUnitTest.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import static org.assertj.core.api.Assertions.*;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.net.URL;
+import java.util.Properties;
+
+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.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests for DistributedSystem class. These tests require file system I/O.
+ */
+@Category(IntegrationTest.class)
+public class DistributedSystemIntegrationJUnitTest {
+
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  
+  @Rule
+  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+  
+  @Rule
+  public final TestName testName = new TestName();
+  
+  @Test
+  public void getPropertiesFileShouldUsePathInSystemProperty() throws Exception {
+    File propertiesFile = this.temporaryFolder.newFile("test.properties");
+    System.setProperty(DistributedSystem.PROPERTIES_FILE_PROPERTY, propertiesFile.getCanonicalPath());
+    Properties properties = new Properties();
+    properties.store(new FileWriter(propertiesFile, false), this.testName.getMethodName());
+
+    assertThat(DistributedSystem.getPropertiesFile()).isEqualTo(propertiesFile.getCanonicalPath());
+  }
+  
+  @Test
+  public void getPropertiesFileUrlShouldUsePathInSystemProperty() throws Exception {
+    File propertiesFile = this.temporaryFolder.newFile("test.properties");
+    System.setProperty(DistributedSystem.PROPERTIES_FILE_PROPERTY, propertiesFile.getCanonicalPath());
+    Properties properties = new Properties();
+    properties.store(new FileWriter(propertiesFile, false), this.testName.getMethodName());
+
+    URL propertiesURL = propertiesFile.getCanonicalFile().toURI().toURL();
+    assertThat(DistributedSystem.getPropertiesFileURL()).isEqualTo(propertiesURL);
+  }
+  
+  @Test
+  public void getSecurityPropertiesFileShouldUsePathInSystemProperty() throws Exception {
+    File propertiesFile = this.temporaryFolder.newFile("testsecurity.properties");
+    System.setProperty(DistributedSystem.SECURITY_PROPERTIES_FILE_PROPERTY, propertiesFile.getCanonicalPath());
+    Properties properties = new Properties();
+    properties.store(new FileWriter(propertiesFile, false), this.testName.getMethodName());
+
+    assertThat(DistributedSystem.getSecurityPropertiesFile()).isEqualTo(propertiesFile.getCanonicalPath());
+  }
+  
+  @Test
+  public void getSecurityPropertiesFileUrlShouldUsePathInSystemProperty() throws Exception {
+    File propertiesFile = this.temporaryFolder.newFile("testsecurity.properties");
+    System.setProperty(DistributedSystem.SECURITY_PROPERTIES_FILE_PROPERTY, propertiesFile.getCanonicalPath());
+    Properties properties = new Properties();
+    properties.store(new FileWriter(propertiesFile, false), this.testName.getMethodName());
+
+    URL propertiesURL = propertiesFile.getCanonicalFile().toURI().toURL();
+    assertThat(DistributedSystem.getSecurityPropertiesFileURL()).isEqualTo(propertiesURL);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemJUnitTest.java
new file mode 100755
index 0000000..56a9646
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/DistributedSystemJUnitTest.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import static org.assertj.core.api.Assertions.*;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+/**
+ * Unit tests for DistributedSystem class.
+ */
+@Category(UnitTest.class)
+public class DistributedSystemJUnitTest {
+  
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  
+  @Test
+  public void getPropertiesFileShouldUseDefault() throws Exception {
+    assertThat(DistributedSystem.getPropertiesFile()).isEqualTo(DistributedSystem.PROPERTIES_FILE_DEFAULT);
+  }
+  
+  @Test
+  public void getPropertiesFileShouldUseSystemProperty() throws Exception {
+    String propertiesFileName = "test.properties";
+    System.setProperty(DistributedSystem.PROPERTIES_FILE_PROPERTY, propertiesFileName);
+    
+    assertThat(DistributedSystem.getPropertiesFile()).isEqualTo(propertiesFileName);
+  }
+  
+  @Test
+  public void getPropertiesFileShouldUseSystemPropertyPath() throws Exception {
+    String propertiesFileName = "/home/test.properties";
+    System.setProperty(DistributedSystem.PROPERTIES_FILE_PROPERTY, propertiesFileName);
+    
+    assertThat(DistributedSystem.getPropertiesFile()).isEqualTo(propertiesFileName);
+  }
+  
+  @Test
+  public void getSecurityPropertiesFileShouldUseDefault() throws Exception {
+    assertThat(DistributedSystem.getSecurityPropertiesFile()).isEqualTo(DistributedSystem.SECURITY_PROPERTIES_FILE_DEFAULT);
+  }
+  
+  @Test
+  public void getSecurityPropertiesFileShouldUseSystemProperty() throws Exception {
+    String propertiesFileName = "testsecurity.properties";
+    System.setProperty(DistributedSystem.SECURITY_PROPERTIES_FILE_PROPERTY, propertiesFileName);
+    
+    assertThat(DistributedSystem.getSecurityPropertiesFile()).isEqualTo(propertiesFileName);
+  }
+  
+  @Test
+  public void getSecurityPropertiesFileShouldUseSystemPropertyPath() throws Exception {
+    String propertiesFileName = "/home/testsecurity.properties";
+    System.setProperty(DistributedSystem.SECURITY_PROPERTIES_FILE_PROPERTY, propertiesFileName);
+    
+    assertThat(DistributedSystem.getSecurityPropertiesFile()).isEqualTo(propertiesFileName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherIntegrationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherIntegrationJUnitTest.java
new file mode 100755
index 0000000..3b56554
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherIntegrationJUnitTest.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import static com.googlecode.catchexception.apis.BDDCatchException.caughtException;
+import static com.googlecode.catchexception.apis.BDDCatchException.when;
+import static org.assertj.core.api.BDDAssertions.*;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Properties;
+
+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.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import com.gemstone.gemfire.distributed.LocatorLauncher.Builder;
+import com.gemstone.gemfire.distributed.LocatorLauncher.Command;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests for LocatorLauncher. These tests require file system I/O.
+ */
+@Category(IntegrationTest.class)
+public class LocatorLauncherIntegrationJUnitTest {
+
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  
+  @Rule
+  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+  
+  @Rule
+  public final TestName testName = new TestName();
+  
+  @Test
+  public void testBuilderParseArgumentsWithValuesSeparatedWithCommas() throws Exception {
+    // given: a new builder and working directory
+    String expectedWorkingDirectory = this.temporaryFolder.getRoot().getCanonicalPath();
+    Builder builder = new Builder();
+
+    // when: parsing many arguments
+    builder.parseArguments(
+        "start", 
+        "memberOne", 
+        "--bind-address", InetAddress.getLocalHost().getHostAddress(),
+        "--dir", expectedWorkingDirectory, 
+        "--hostname-for-clients", "Tucows", 
+        "--pid", "1234", 
+        "--port", "11235",
+        "--redirect-output", 
+        "--force", 
+        "--debug");
+
+    // then: the getters should return properly parsed values
+    assertThat(builder.getCommand()).isEqualTo(Command.START);
+    assertThat(builder.getBindAddress()).isEqualTo(InetAddress.getLocalHost());
+    assertThat(builder.getWorkingDirectory()).isEqualTo(expectedWorkingDirectory);
+    assertThat(builder.getHostnameForClients()).isEqualTo("Tucows");
+    assertThat(builder.getPid().intValue()).isEqualTo(1234);
+    assertThat(builder.getPort().intValue()).isEqualTo(11235);
+    assertThat(builder.getRedirectOutput()).isTrue();
+    assertThat(builder.getForce()).isTrue();
+    assertThat(builder.getDebug()).isTrue();
+  }
+
+  @Test
+  public void testBuilderParseArgumentsWithValuesSeparatedWithEquals() throws Exception {
+    // given: a new builder and a directory
+    String expectedWorkingDirectory = this.temporaryFolder.getRoot().getCanonicalPath();
+    Builder builder = new Builder();
+
+    // when: parsing arguments with values separated by equals
+    builder.parseArguments(
+        "start", 
+        "--dir=" + expectedWorkingDirectory, 
+        "--port=" + "12345", 
+        "memberOne");
+
+    // then: the getters should return properly parsed values
+    assertThat(builder.getCommand()).isEqualTo(Command.START);
+    assertThat(builder.getDebug()).isFalse();
+    assertThat(builder.getForce()).isFalse();
+    assertThat(builder.getHelp()).isFalse();
+    assertThat(builder.getBindAddress()).isNull();
+    assertThat(builder.getHostnameForClients()).isNull();
+    assertThat(builder.getMemberName()).isEqualTo("memberOne");
+    assertThat(builder.getPid()).isNull();
+    assertThat(builder.getWorkingDirectory()).isEqualTo(expectedWorkingDirectory);
+    assertThat(builder.getPort().intValue()).isEqualTo(12345);
+  }
+
+  @Test
+  public void testBuildWithMemberNameSetInGemFirePropertiesOnStart() throws Exception {
+    // given: gemfire.properties with a name
+    Properties gemfireProperties = new Properties();
+    gemfireProperties.setProperty(DistributionConfig.NAME_NAME, "locator123");
+    useGemFirePropertiesFileInTemporaryFolder("gemfire.properties", gemfireProperties);
+    
+    // when: starting with null MemberName
+    LocatorLauncher launcher = new Builder()
+        .setCommand(Command.START)
+        .setMemberName(null)
+        .build();
+
+    // then: name in gemfire.properties file should be used for MemberName
+    assertThat(launcher).isNotNull();
+    assertThat(launcher.getCommand()).isEqualTo(Command.START);
+    assertThat(launcher.getMemberName()).isNull();
+  }
+
+  @Test
+  public void testBuildWithNoMemberNameOnStart() throws Exception {
+    // given: gemfire.properties with no name
+    useGemFirePropertiesFileInTemporaryFolder("gemfire.properties", new Properties());
+
+    // when: no MemberName is specified
+    when(new Builder()
+        .setCommand(Command.START))
+        .build();
+    
+    // then: throw IllegalStateException
+    then(caughtException())
+        .isExactlyInstanceOf(IllegalStateException.class)
+        .hasMessage(LocalizedStrings.Launcher_Builder_MEMBER_NAME_VALIDATION_ERROR_MESSAGE.toLocalizedString("Locator"));
+  }
+
+  @Test
+  public void testBuilderSetAndGetWorkingDirectory() throws Exception {
+    // given: a new builder and a directory
+    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+    Builder builder = new Builder();
+
+    // when: not setting WorkingDirectory
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+    
+    // when: setting WorkingDirectory to null
+    assertThat(builder.setWorkingDirectory(null)).isSameAs(builder);
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+
+    // when: setting WorkingDirectory to empty string
+    assertThat(builder.setWorkingDirectory("")).isSameAs(builder);
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+
+    // when: setting WorkingDirectory to white space
+    assertThat(builder.setWorkingDirectory("  ")).isSameAs(builder);
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+
+    // when: setting WorkingDirectory to a directory
+    assertThat(builder.setWorkingDirectory(rootFolder)).isSameAs(builder);
+    // then: getWorkingDirectory returns that directory
+    assertThat(builder.getWorkingDirectory()).isEqualTo(rootFolder);
+
+    // when: setting WorkingDirectory to null (again)
+    assertThat(builder.setWorkingDirectory(null)).isSameAs(builder);
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+  }
+
+  @Test
+  public void testBuilderSetWorkingDirectoryToFile() throws IOException {
+    // given: a file instead of a directory
+    File tmpFile = this.temporaryFolder.newFile();
+
+    // when: setting WorkingDirectory to that file
+    when(new Builder())
+        .setWorkingDirectory(tmpFile.getCanonicalPath());
+
+    // then: throw IllegalArgumentException
+    then(caughtException())
+        .isExactlyInstanceOf(IllegalArgumentException.class)
+        .hasMessage(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE.toLocalizedString("Locator"))
+        .hasCause(new FileNotFoundException(tmpFile.getCanonicalPath()));
+  }
+
+  @Test
+  public void testBuildSetWorkingDirectoryToNonCurrentDirectoryOnStart() throws Exception {
+    // given: using LocatorLauncher in-process
+    
+    // when: setting WorkingDirectory to non-current directory
+    when(new Builder()
+        .setCommand(Command.START)
+        .setMemberName("memberOne")
+        .setWorkingDirectory(this.temporaryFolder.getRoot().getCanonicalPath()))
+        .build();
+    
+    // then: throw IllegalStateException
+    then(caughtException())
+        .isExactlyInstanceOf(IllegalStateException.class)
+        .hasMessage(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_OPTION_NOT_VALID_ERROR_MESSAGE.toLocalizedString("Locator"));
+  }
+  
+  @Test
+  public void testBuilderSetWorkingDirectoryToNonExistingDirectory() {
+    // when: setting WorkingDirectory to non-existing directory
+    when(new Builder())
+        .setWorkingDirectory("/path/to/non_existing/directory");
+    
+    // then: throw IllegalArgumentException
+    then(caughtException())
+        .isExactlyInstanceOf(IllegalArgumentException.class)
+        .hasMessage(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE.toLocalizedString("Locator"))
+        .hasCause(new FileNotFoundException("/path/to/non_existing/directory"));
+  }
+
+  /**
+   * Creates a gemfire properties file in temporaryFolder:
+   * <ol>
+   * <li>creates <code>fileName</code> in <code>temporaryFolder</code></li>
+   * <li>sets "gemfirePropertyFile" system property</li>
+   * <li>writes <code>gemfireProperties</code> to the file</li>
+   * </ol>
+   */
+  private void useGemFirePropertiesFileInTemporaryFolder(final String fileName, final Properties gemfireProperties) throws Exception {
+    File propertiesFile = new File(this.temporaryFolder.getRoot().getCanonicalPath(), fileName);
+    System.setProperty(DistributedSystem.PROPERTIES_FILE_PROPERTY, propertiesFile.getCanonicalPath());
+    
+    gemfireProperties.store(new FileWriter(propertiesFile, false), this.testName.getMethodName());
+    assertThat(propertiesFile.isFile()).isTrue();
+    assertThat(propertiesFile.exists()).isTrue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java
index 003a098..c1b6ed0 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherJUnitTest.java
@@ -17,14 +17,9 @@
 package com.gemstone.gemfire.distributed;
 
 import static org.junit.Assert.*;
-import static org.junit.Assume.*;
 
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
-import java.util.Properties;
 
 import com.gemstone.gemfire.distributed.LocatorLauncher.Builder;
 import com.gemstone.gemfire.distributed.LocatorLauncher.Command;
@@ -34,8 +29,11 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 import joptsimple.OptionException;
 
+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.rules.TestName;
 
 /**
  * The LocatorLauncherJUnitTest class is a test suite of test cases for testing the contract and functionality of
@@ -52,46 +50,13 @@ import org.junit.experimental.categories.Category;
  * @since 7.0
  */
 @Category(UnitTest.class)
-public class LocatorLauncherJUnitTest extends CommonLauncherTestSuite {
+public class LocatorLauncherJUnitTest {
 
-  @Test
-  public void testBuilderParseArguments() throws Exception {
-    String expectedWorkingDirectory = this.temporaryFolder.getRoot().getCanonicalPath().toString();
-    Builder builder = new Builder();
-
-    builder.parseArguments("start", "memberOne", "--bind-address", InetAddress.getLocalHost().getHostAddress(),
-      "--dir", expectedWorkingDirectory, "--hostname-for-clients", "Tucows", "--pid", "1234", "--port", "11235",
-        "--redirect-output", "--force", "--debug");
-
-    assertEquals(Command.START, builder.getCommand());
-    assertEquals(InetAddress.getLocalHost(), builder.getBindAddress());
-    assertEquals(expectedWorkingDirectory, builder.getWorkingDirectory());
-    assertEquals("Tucows", builder.getHostnameForClients());
-    assertEquals(1234, builder.getPid().intValue());
-    assertEquals(11235, builder.getPort().intValue());
-    assertTrue(builder.getRedirectOutput());
-    assertTrue(builder.getForce());
-    assertTrue(builder.getDebug());
-  }
-
-  @Test
-  public void testBuilderParseArgumentsWithCommandInArguments() throws Exception {
-    String expectedWorkingDirectory = this.temporaryFolder.getRoot().getCanonicalPath().toString();
-    Builder builder = new Builder();
-
-    builder.parseArguments("start", "--dir=" + expectedWorkingDirectory, "--port", "12345", "memberOne");
-
-    assertEquals(Command.START, builder.getCommand());
-    assertFalse(Boolean.TRUE.equals(builder.getDebug()));
-    assertFalse(Boolean.TRUE.equals(builder.getForce()));
-    assertFalse(Boolean.TRUE.equals(builder.getHelp()));
-    assertNull(builder.getBindAddress());
-    assertNull(builder.getHostnameForClients());
-    assertEquals("12345", builder.getMemberName());
-    assertNull(builder.getPid());
-    assertEquals(expectedWorkingDirectory, builder.getWorkingDirectory());
-    assertEquals(12345, builder.getPort().intValue());
-  }
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  
+  @Rule
+  public final TestName testName = new TestName();
 
   @Test(expected = IllegalArgumentException.class)
   public void testBuilderParseArgumentsWithNonNumericPort() {
@@ -307,59 +272,6 @@ public class LocatorLauncherJUnitTest extends CommonLauncherTestSuite {
   }
 
   @Test
-  public void testSetAndGetWorkingDirectory() throws Exception {
-    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath().toString();
-    Builder builder = new Builder();
-
-    assertEquals(AbstractLauncher.DEFAULT_WORKING_DIRECTORY, builder.getWorkingDirectory());
-    assertSame(builder, builder.setWorkingDirectory(null));
-    assertEquals(AbstractLauncher.DEFAULT_WORKING_DIRECTORY, builder.getWorkingDirectory());
-    assertSame(builder, builder.setWorkingDirectory(""));
-    assertEquals(AbstractLauncher.DEFAULT_WORKING_DIRECTORY, builder.getWorkingDirectory());
-    assertSame(builder, builder.setWorkingDirectory("  "));
-    assertEquals(AbstractLauncher.DEFAULT_WORKING_DIRECTORY, builder.getWorkingDirectory());
-    assertSame(builder, builder.setWorkingDirectory(rootFolder));
-    assertEquals(rootFolder, builder.getWorkingDirectory());
-    assertSame(builder, builder.setWorkingDirectory(null));
-    assertEquals(AbstractLauncher.DEFAULT_WORKING_DIRECTORY, builder.getWorkingDirectory());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testSetWorkingDirectoryToFile() throws IOException {
-    File tmpFile = File.createTempFile("tmp", "file");
-
-    assertNotNull(tmpFile);
-    assertTrue(tmpFile.isFile());
-
-    tmpFile.deleteOnExit();
-
-    try {
-      new Builder().setWorkingDirectory(tmpFile.getCanonicalPath());
-    }
-    catch (IllegalArgumentException expected) {
-      assertEquals(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE
-        .toLocalizedString("Locator"), expected.getMessage());
-      assertTrue(expected.getCause() instanceof FileNotFoundException);
-      assertEquals(tmpFile.getCanonicalPath(), expected.getCause().getMessage());
-      throw expected;
-    }
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testSetWorkingDirectoryToNonExistingDirectory() {
-    try {
-      new Builder().setWorkingDirectory("/path/to/non_existing/directory");
-    }
-    catch (IllegalArgumentException expected) {
-      assertEquals(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE
-        .toLocalizedString("Locator"), expected.getMessage());
-      assertTrue(expected.getCause() instanceof FileNotFoundException);
-      assertEquals("/path/to/non_existing/directory", expected.getCause().getMessage());
-      throw expected;
-    }
-  }
-
-  @Test
   public void testBuild() throws Exception {
     Builder builder = new Builder();
 
@@ -396,26 +308,6 @@ public class LocatorLauncherJUnitTest extends CommonLauncherTestSuite {
   }
 
   @Test
-  public void testBuildWithMemberNameSetInGemfirePropertiesOnStart() throws Exception {
-    System.setProperty("user.dir", this.temporaryFolder.getRoot().getCanonicalPath());
-    
-    Properties gemfireProperties = new Properties();
-    gemfireProperties.setProperty(DistributionConfig.NAME_NAME, "locator123");
-    File gemfirePropertiesFile = writeGemFirePropertiesToFile(gemfireProperties, "gemfire.properties",
-      String.format("Test gemfire.properties file for %1$s.%2$s.", getClass().getSimpleName(),
-        "testBuildWithMemberNameSetInGemfirePropertiesOnStart"));
-
-    assertNotNull(gemfirePropertiesFile);
-    assertTrue(gemfirePropertiesFile.isFile());
-
-    LocatorLauncher launcher = new Builder().setCommand(Command.START).setMemberName(null).build();
-
-    assertNotNull(launcher);
-    assertEquals(Command.START, launcher.getCommand());
-    assertNull(launcher.getMemberName());
-  }
-
-  @Test
   public void testBuildWithMemberNameSetInSystemPropertiesOnStart() {
     System.setProperty(DistributionConfig.GEMFIRE_PREFIX + DistributionConfig.NAME_NAME, "locatorXYZ");
 
@@ -428,32 +320,4 @@ public class LocatorLauncherJUnitTest extends CommonLauncherTestSuite {
     assertEquals(LocatorLauncher.Command.START, launcher.getCommand());
     assertNull(launcher.getMemberName());
   }
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildWithNoMemberNameOnStart() throws Exception {
-    System.setProperty("user.dir", this.temporaryFolder.getRoot().getCanonicalPath());
-    try {
-      new Builder().setCommand(Command.START).build();
-    }
-    catch (IllegalStateException expected) {
-      assertEquals(LocalizedStrings.Launcher_Builder_MEMBER_NAME_VALIDATION_ERROR_MESSAGE.toLocalizedString("Locator"),
-        expected.getMessage());
-      throw expected;
-    }
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void testBuildWithMismatchingCurrentAndWorkingDirectoryOnStart() throws Exception {
-    try {
-      new Builder().setCommand(Command.START)
-        .setMemberName("memberOne")
-        .setWorkingDirectory(this.temporaryFolder.getRoot().getCanonicalPath().toString())
-        .build();
-    }
-    catch (IllegalStateException expected) {
-      assertEquals(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_OPTION_NOT_VALID_ERROR_MESSAGE
-        .toLocalizedString("Locator"), expected.getMessage());
-      throw expected;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherIntegrationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherIntegrationJUnitTest.java
new file mode 100755
index 0000000..b61f89d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherIntegrationJUnitTest.java
@@ -0,0 +1,312 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.distributed;
+
+import static com.googlecode.catchexception.apis.BDDCatchException.caughtException;
+import static com.googlecode.catchexception.apis.BDDCatchException.when;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.BDDAssertions.*;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileWriter;
+import java.net.InetAddress;
+import java.util.Properties;
+
+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.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import com.gemstone.gemfire.distributed.ServerLauncher.Builder;
+import com.gemstone.gemfire.distributed.ServerLauncher.Command;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests for ServerLauncher class. These tests may require file system and/or network I/O.
+ */
+@Category(IntegrationTest.class)
+public class ServerLauncherIntegrationJUnitTest {
+
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  
+  @Rule
+  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+  
+  @Rule
+  public final TestName testName = new TestName();
+  
+  @Test
+  public void testBuildWithManyArguments() throws Exception {
+    // given
+    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+    
+    // when
+    ServerLauncher launcher = new Builder()
+        .setCommand(Command.STOP)
+        .setAssignBuckets(true)
+        .setForce(true)
+        .setMemberName("serverOne")
+        .setRebalance(true)
+        .setServerBindAddress(InetAddress.getLocalHost().getHostAddress())
+        .setServerPort(11235)
+        .setWorkingDirectory(rootFolder)
+        .setCriticalHeapPercentage(90.0f)
+        .setEvictionHeapPercentage(75.0f)
+        .setMaxConnections(100)
+        .setMaxMessageCount(512)
+        .setMaxThreads(8)
+        .setMessageTimeToLive(120000)
+        .setSocketBufferSize(32768)
+        .build();
+
+    // then
+    assertThat(launcher).isNotNull();
+    assertThat(launcher.isAssignBuckets()).isTrue();
+    assertThat(launcher.isDebugging()).isFalse();
+    assertThat(launcher.isDisableDefaultServer()).isFalse();
+    assertThat(launcher.isForcing()).isTrue();
+    assertThat(launcher.isHelping()).isFalse();
+    assertThat(launcher.isRebalancing()).isTrue();
+    assertThat(launcher.isRunning()).isFalse();
+    assertThat(launcher.getCommand()).isEqualTo(Command.STOP);
+    assertThat(launcher.getMemberName()).isEqualTo("serverOne");
+    assertThat(launcher.getServerBindAddress()).isEqualTo(InetAddress.getLocalHost());
+    assertThat(launcher.getServerPort().intValue()).isEqualTo(11235);
+    assertThat(launcher.getWorkingDirectory()).isEqualTo(rootFolder);
+    assertThat(launcher.getCriticalHeapPercentage().floatValue()).isEqualTo(90.0f);
+    assertThat(launcher.getEvictionHeapPercentage().floatValue()).isEqualTo(75.0f);
+    assertThat(launcher.getMaxConnections().intValue()).isEqualTo(100);
+    assertThat(launcher.getMaxMessageCount().intValue()).isEqualTo(512);
+    assertThat(launcher.getMaxThreads().intValue()).isEqualTo(8);
+    assertThat(launcher.getMessageTimeToLive().intValue()).isEqualTo(120000);
+    assertThat(launcher.getSocketBufferSize().intValue()).isEqualTo(32768);
+  }
+
+  @Test
+  public void testBuilderParseArgumentsWithValuesSeparatedWithCommas() throws Exception {
+    // given a new builder and a directory
+    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+    Builder builder = new Builder();
+
+    // when: parsing many arguments
+    builder.parseArguments(
+        "start", 
+        "serverOne", 
+        "--assign-buckets", 
+        "--disable-default-server", 
+        "--debug", 
+        "--force",
+        "--rebalance", 
+        "--redirect-output", 
+        "--dir", rootFolder, 
+        "--pid", "1234",
+        "--server-bind-address", InetAddress.getLocalHost().getHostAddress(), 
+        "--server-port", "11235", 
+        "--hostname-for-clients", "192.168.99.100");
+
+    // then: the getters should return properly parsed values
+    assertThat(builder.getCommand()).isEqualTo(Command.START);
+    assertThat(builder.getMemberName()).isEqualTo("serverOne");
+    assertThat(builder.getHostNameForClients()).isEqualTo("192.168.99.100");
+    assertThat(builder.getAssignBuckets()).isTrue();
+    assertThat(builder.getDisableDefaultServer()).isTrue();
+    assertThat(builder.getDebug()).isTrue();
+    assertThat(builder.getForce()).isTrue();
+    assertThat(builder.getHelp()).isFalse();
+    assertThat(builder.getRebalance()).isTrue();
+    assertThat(builder.getRedirectOutput()).isTrue();
+    assertThat(builder.getWorkingDirectory()).isEqualTo(rootFolder);
+    assertThat(builder.getPid().intValue()).isEqualTo(1234);
+    assertThat(builder.getServerBindAddress()).isEqualTo(InetAddress.getLocalHost());
+    assertThat(builder.getServerPort().intValue()).isEqualTo(11235);
+  }
+
+  @Test
+  public void testBuilderParseArgumentsWithValuesSeparatedWithEquals() throws Exception {
+    // given a new builder and a directory
+    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+    Builder builder = new Builder();
+
+    // when: parsing many arguments
+    builder.parseArguments(
+        "start", 
+        "serverOne", 
+        "--assign-buckets", 
+        "--disable-default-server", 
+        "--debug", 
+        "--force",
+        "--rebalance", 
+        "--redirect-output", 
+        "--dir=" + rootFolder, 
+        "--pid=1234",
+        "--server-bind-address=" + InetAddress.getLocalHost().getHostAddress(), 
+        "--server-port=11235", 
+        "--hostname-for-clients=192.168.99.100");
+
+    // then: the getters should return properly parsed values
+    assertThat(builder.getCommand()).isEqualTo(Command.START);
+    assertThat(builder.getMemberName()).isEqualTo("serverOne");
+    assertThat(builder.getHostNameForClients()).isEqualTo("192.168.99.100");
+    assertThat(builder.getAssignBuckets()).isTrue();
+    assertThat(builder.getDisableDefaultServer()).isTrue();
+    assertThat(builder.getDebug()).isTrue();
+    assertThat(builder.getForce()).isTrue();
+    assertThat(builder.getHelp()).isFalse();
+    assertThat(builder.getRebalance()).isTrue();
+    assertThat(builder.getRedirectOutput()).isTrue();
+    assertThat(builder.getWorkingDirectory()).isEqualTo(rootFolder);
+    assertThat(builder.getPid().intValue()).isEqualTo(1234);
+    assertThat(builder.getServerBindAddress()).isEqualTo(InetAddress.getLocalHost());
+    assertThat(builder.getServerPort().intValue()).isEqualTo(11235);
+  }
+
+  @Test
+  public void testBuildWithMemberNameSetInGemFirePropertiesOnStart() throws Exception {
+    // given: gemfire.properties with a name
+    Properties gemfireProperties = new Properties();
+    gemfireProperties.setProperty(DistributionConfig.NAME_NAME, "server123");
+    useGemFirePropertiesFileInTemporaryFolder("gemfire.properties", gemfireProperties);
+
+    // when: starting with null MemberName
+    ServerLauncher launcher = new Builder()
+        .setCommand(Command.START)
+        .setMemberName(null)
+        .build();
+
+    // then: name in gemfire.properties file should be used for MemberName
+    assertThat(launcher).isNotNull();
+    assertThat(launcher.getCommand()).isEqualTo(Command.START);
+    assertThat(launcher.getMemberName()).isNull();
+  }
+  
+  @Test
+  public void testBuildWithNoMemberNameOnStart() throws Exception {
+    // given: gemfire.properties with no name
+    useGemFirePropertiesFileInTemporaryFolder("gemfire.properties", new Properties());
+
+    // when: no MemberName is specified
+    when(new Builder()
+        .setCommand(Command.START))
+        .build();
+    
+    // then: throw IllegalStateException
+    then(caughtException())
+        .isExactlyInstanceOf(IllegalStateException.class)
+        .hasMessage(LocalizedStrings.Launcher_Builder_MEMBER_NAME_VALIDATION_ERROR_MESSAGE.toLocalizedString("Server"));
+  }
+
+  @Test
+  public void testBuilderSetAndGetWorkingDirectory() throws Exception {
+    // given: a new builder and a directory
+    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath();
+    Builder builder = new Builder();
+
+    // when: not setting WorkingDirectory
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(ServerLauncher.DEFAULT_WORKING_DIRECTORY);
+    
+    // when: setting WorkingDirectory to null
+    assertThat(builder.setWorkingDirectory(null)).isSameAs(builder);
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+
+    // when: setting WorkingDirectory to empty string
+    assertThat(builder.setWorkingDirectory("")).isSameAs(builder);
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+
+    // when: setting WorkingDirectory to white space
+    assertThat(builder.setWorkingDirectory("  ")).isSameAs(builder);
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+
+    // when: setting WorkingDirectory to a directory
+    assertThat(builder.setWorkingDirectory(rootFolder)).isSameAs(builder);
+    // then: getWorkingDirectory returns that directory
+    assertThat(builder.getWorkingDirectory()).isEqualTo(rootFolder);
+
+    // when: setting WorkingDirectory to null (again)
+    assertThat(builder.setWorkingDirectory(null)).isSameAs(builder);
+    // then: getWorkingDirectory returns default
+    assertThat(builder.getWorkingDirectory()).isEqualTo(AbstractLauncher.DEFAULT_WORKING_DIRECTORY);
+  }
+
+  @Test
+  public void testBuilderSetWorkingDirectoryToFile() throws Exception {
+    // given: a file instead of a directory
+    File tmpFile = this.temporaryFolder.newFile();
+
+    // when: setting WorkingDirectory to that file
+    when(new Builder())
+        .setWorkingDirectory(tmpFile.getAbsolutePath());
+    
+    // then: throw IllegalArgumentException
+    then(caughtException())
+        .hasMessage(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE.toLocalizedString("Server"))
+        .hasCause(new FileNotFoundException(tmpFile.getAbsolutePath()));
+  }
+
+  @Test
+  public void testBuildSetWorkingDirectoryToNonCurrentDirectoryOnStart() throws Exception {
+    // given: using ServerLauncher in-process
+
+    // when: setting WorkingDirectory to non-current directory
+    when(new Builder()
+        .setCommand(Command.START)
+        .setMemberName("serverOne")
+        .setWorkingDirectory(this.temporaryFolder.getRoot().getCanonicalPath()))
+        .build();
+    
+    // then: throw IllegalStateException
+    then(caughtException())
+        .isExactlyInstanceOf(IllegalStateException.class)
+        .hasMessage(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_OPTION_NOT_VALID_ERROR_MESSAGE.toLocalizedString("Server"));
+  }
+
+  @Test
+  public void testBuilderSetWorkingDirectoryToNonExistingDirectory() {
+    // when: setting WorkingDirectory to non-existing directory
+    when(new Builder())
+        .setWorkingDirectory("/path/to/non_existing/directory");
+    
+    // then: throw IllegalArgumentException
+    then(caughtException())
+        .hasMessage(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE.toLocalizedString("Server"))
+        .hasCause(new FileNotFoundException("/path/to/non_existing/directory"));
+  }
+
+  /**
+   * Creates a gemfire properties file in temporaryFolder:
+   * <li>creates <code>fileName</code> in <code>temporaryFolder</code>
+   * <li>sets "gemfirePropertyFile" system property
+   * <li>writes <code>gemfireProperties</code> to the file
+   */
+  private void useGemFirePropertiesFileInTemporaryFolder(final String fileName, final Properties gemfireProperties) throws Exception {
+    File propertiesFile = new File(this.temporaryFolder.getRoot().getCanonicalPath(), fileName);
+    System.setProperty(DistributedSystem.PROPERTIES_FILE_PROPERTY, propertiesFile.getCanonicalPath());
+    
+    gemfireProperties.store(new FileWriter(propertiesFile, false), this.testName.getMethodName());
+    assertThat(propertiesFile.isFile()).isTrue();
+    assertThat(propertiesFile.exists()).isTrue();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java
index 6884e58..802df9f 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherJUnitTest.java
@@ -46,8 +46,11 @@ import org.jmock.lib.legacy.ClassImposteriser;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
+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.rules.TestName;
 
 /**
  * The ServerLauncherJUnitTest class is a test suite of unit tests testing the contract, functionality and invariants
@@ -65,10 +68,16 @@ import org.junit.experimental.categories.Category;
  */
 @SuppressWarnings({"deprecation", "unused"})
 @Category(UnitTest.class)
-public class ServerLauncherJUnitTest extends CommonLauncherTestSuite {
+public class ServerLauncherJUnitTest {
 
   private Mockery mockContext;
 
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  
+  @Rule
+  public final TestName testName = new TestName();
+  
   @Before
   public void setup() {
     mockContext = new Mockery() {{
@@ -84,31 +93,6 @@ public class ServerLauncherJUnitTest extends CommonLauncherTestSuite {
   }
 
   @Test
-  public void testParseArguments() throws Exception {
-    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath().toString();
-    Builder builder = new Builder();
-
-    builder.parseArguments("start", "serverOne", "--assign-buckets", "--disable-default-server", "--debug", "--force",
-      "--rebalance", "--redirect-output", "--dir=" + rootFolder, "--pid=1234",
-        "--server-bind-address=" + InetAddress.getLocalHost().getHostAddress(), "--server-port=11235", "--hostname-for-clients=192.168.99.100");
-
-    assertEquals(Command.START, builder.getCommand());
-    assertEquals("serverOne", builder.getMemberName());
-    assertEquals("192.168.99.100", builder.getHostNameForClients());
-    assertTrue(builder.getAssignBuckets());
-    assertTrue(builder.getDisableDefaultServer());
-    assertTrue(builder.getDebug());
-    assertTrue(builder.getForce());
-    assertFalse(Boolean.TRUE.equals(builder.getHelp()));
-    assertTrue(builder.getRebalance());
-    assertTrue(builder.getRedirectOutput());
-    assertEquals(rootFolder, builder.getWorkingDirectory());
-    assertEquals(1234, builder.getPid().intValue());
-    assertEquals(InetAddress.getLocalHost(), builder.getServerBindAddress());
-    assertEquals(11235, builder.getServerPort().intValue());
-  }
-
-  @Test
   public void testParseCommand() {
     Builder builder = new Builder();
 
@@ -323,57 +307,6 @@ public class ServerLauncherJUnitTest extends CommonLauncherTestSuite {
   }
 
   @Test
-  public void testSetAndGetWorkingDirectory() throws Exception {
-    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath().toString();
-    Builder builder = new Builder();
-
-    assertEquals(ServerLauncher.DEFAULT_WORKING_DIRECTORY, builder.getWorkingDirectory());
-    assertSame(builder, builder.setWorkingDirectory(rootFolder));
-    assertEquals(rootFolder, builder.getWorkingDirectory());
-    assertSame(builder, builder.setWorkingDirectory("  "));
-    assertEquals(ServerLauncher.DEFAULT_WORKING_DIRECTORY, builder.getWorkingDirectory());
-    assertSame(builder, builder.setWorkingDirectory(""));
-    assertEquals(ServerLauncher.DEFAULT_WORKING_DIRECTORY, builder.getWorkingDirectory());
-    assertSame(builder, builder.setWorkingDirectory(null));
-    assertEquals(ServerLauncher.DEFAULT_WORKING_DIRECTORY, builder.getWorkingDirectory());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testSetWorkingDirectoryToNonExistingDirectory() {
-    try {
-      new Builder().setWorkingDirectory("/path/to/non_existing/directory");
-    }
-    catch (IllegalArgumentException expected) {
-      assertEquals(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE
-        .toLocalizedString("Server"), expected.getMessage());
-      assertTrue(expected.getCause() instanceof FileNotFoundException);
-      assertEquals("/path/to/non_existing/directory", expected.getCause().getMessage());
-      throw expected;
-    }
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testSetWorkingDirectoryToFile() throws IOException {
-    File tmpFile = File.createTempFile("tmp", "file");
-
-    assertNotNull(tmpFile);
-    assertTrue(tmpFile.isFile());
-
-    tmpFile.deleteOnExit();
-
-    try {
-      new Builder().setWorkingDirectory(tmpFile.getAbsolutePath());
-    }
-    catch (IllegalArgumentException expected) {
-      assertEquals(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_NOT_FOUND_ERROR_MESSAGE
-        .toLocalizedString("Server"), expected.getMessage());
-      assertTrue(expected.getCause() instanceof FileNotFoundException);
-      assertEquals(tmpFile.getAbsolutePath(), expected.getCause().getMessage());
-      throw expected;
-    }
-  }
-
-  @Test
   public void testSetAndGetCriticalHeapPercentage() {
     Builder builder = new Builder();
 
@@ -550,50 +483,6 @@ public class ServerLauncherJUnitTest extends CommonLauncherTestSuite {
   }
 
   @Test
-  public void testBuild() throws Exception {
-    String rootFolder = this.temporaryFolder.getRoot().getCanonicalPath().toString();
-    
-    ServerLauncher launcher = new Builder()
-      .setCommand(Command.STOP)
-      .setAssignBuckets(true)
-      .setForce(true)
-      .setMemberName("serverOne")
-      .setRebalance(true)
-      .setServerBindAddress(InetAddress.getLocalHost().getHostAddress())
-      .setServerPort(11235)
-      .setWorkingDirectory(rootFolder)
-      .setCriticalHeapPercentage(90.0f)
-      .setEvictionHeapPercentage(75.0f)
-      .setMaxConnections(100)
-      .setMaxMessageCount(512)
-      .setMaxThreads(8)
-      .setMessageTimeToLive(120000)
-      .setSocketBufferSize(32768)
-      .build();
-
-    assertNotNull(launcher);
-    assertTrue(launcher.isAssignBuckets());
-    assertFalse(launcher.isDebugging());
-    assertFalse(launcher.isDisableDefaultServer());
-    assertTrue(launcher.isForcing());
-    assertFalse(launcher.isHelping());
-    assertTrue(launcher.isRebalancing());
-    assertFalse(launcher.isRunning());
-    assertEquals(Command.STOP, launcher.getCommand());
-    assertEquals("serverOne", launcher.getMemberName());
-    assertEquals(InetAddress.getLocalHost(), launcher.getServerBindAddress());
-    assertEquals(11235, launcher.getServerPort().intValue());
-    assertEquals(rootFolder, launcher.getWorkingDirectory());
-    assertEquals(90.0f, launcher.getCriticalHeapPercentage().floatValue(), 0.0f);
-    assertEquals(75.0f, launcher.getEvictionHeapPercentage().floatValue(), 0.0f);
-    assertEquals(100, launcher.getMaxConnections().intValue());
-    assertEquals(512, launcher.getMaxMessageCount().intValue());
-    assertEquals(8, launcher.getMaxThreads().intValue());
-    assertEquals(120000, launcher.getMessageTimeToLive().intValue());
-    assertEquals(32768, launcher.getSocketBufferSize().intValue());
-  }
-
-  @Test
   public void testBuildWithMemberNameSetInApiPropertiesOnStart() {
     ServerLauncher launcher = new Builder()
       .setCommand(ServerLauncher.Command.START)
@@ -607,24 +496,10 @@ public class ServerLauncherJUnitTest extends CommonLauncherTestSuite {
     assertEquals("serverABC", launcher.getProperties().getProperty(DistributionConfig.NAME_NAME));
   }
 
-  @Ignore("GEODE-69: We need to change DistributedSystem to not use static final for gemfirePropertyFile")
   @Test
-  public void testBuildWithMemberNameSetInGemFirePropertiesOnStart() {
-    Properties gemfireProperties = new Properties();
-
-    gemfireProperties.setProperty(DistributionConfig.NAME_NAME, "server123");
-
-    File gemfirePropertiesFile = writeGemFirePropertiesToFile(
-        gemfireProperties, 
-        "gemfire.properties",
-        String.format("Test gemfire.properties file for %1$s.%2$s.", getClass().getSimpleName(), this.testName.getMethodName()));
-
-    assertNotNull(gemfirePropertiesFile);
-    assertTrue(gemfirePropertiesFile.isFile());
+  public void testBuildWithMemberNameSetInSystemPropertiesOnStart() {
+    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + DistributionConfig.NAME_NAME, "serverXYZ");
 
-    System.setProperty("gemfirePropertyFile", gemfirePropertiesFile.getAbsolutePath());
-    System.out.println("gemfirePropertiesFile.getAbsolutePath()=" + gemfirePropertiesFile.getAbsolutePath());
-    
     ServerLauncher launcher = new Builder()
       .setCommand(ServerLauncher.Command.START)
       .setMemberName(null)
@@ -635,25 +510,6 @@ public class ServerLauncherJUnitTest extends CommonLauncherTestSuite {
     assertNull(launcher.getMemberName());
   }
 
-  @Test
-  public void testBuildWithMemberNameSetInSystemPropertiesOnStart() {
-    try {
-      System.setProperty(DistributionConfig.GEMFIRE_PREFIX + DistributionConfig.NAME_NAME, "serverXYZ");
-
-      ServerLauncher launcher = new Builder()
-        .setCommand(ServerLauncher.Command.START)
-        .setMemberName(null)
-        .build();
-
-      assertNotNull(launcher);
-      assertEquals(ServerLauncher.Command.START, launcher.getCommand());
-      assertNull(launcher.getMemberName());
-    }
-    finally {
-      System.clearProperty(DistributionConfig.GEMFIRE_PREFIX + DistributionConfig.NAME_NAME);
-    }
-  }
-
   @Test(expected = IllegalStateException.class)
   public void testBuildNoMemberNameOnStart() {
     try {
@@ -666,21 +522,6 @@ public class ServerLauncherJUnitTest extends CommonLauncherTestSuite {
     }
   }
 
-  @Test(expected = IllegalStateException.class)
-  public void testBuildWithInvalidWorkingDirectoryOnStart() throws Exception {
-    try {
-      new Builder().setCommand(Command.START)
-        .setMemberName("serverOne")
-        .setWorkingDirectory(this.temporaryFolder.getRoot().getCanonicalPath().toString())
-        .build();
-    }
-    catch (IllegalStateException expected) {
-      assertEquals(LocalizedStrings.Launcher_Builder_WORKING_DIRECTORY_OPTION_NOT_VALID_ERROR_MESSAGE
-        .toLocalizedString("Server"), expected.getMessage());
-      throw expected;
-    }
-  }
-
   @Test
   public void testIsServing() {
     final Cache mockCache = mockContext.mock(Cache.class, "Cache");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessOutputReader.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessOutputReader.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessOutputReader.java
index e99dceb..a2f02c4 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessOutputReader.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessOutputReader.java
@@ -16,83 +16,74 @@
  */
 package com.gemstone.gemfire.test.process;
 
-import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 /**
- * Reads the stdout and stderr from a running process and stores then for test 
- * validation. Also provides a mechanism to waitFor the process to terminate. 
+ * Starts the stdout and stderr reader threads for a running process. Provides
+ * a mechanism to waitFor the process to terminate.
+ * </p>
  * Extracted from ProcessWrapper.
  * 
  * @author Kirk Lund
  */
 public class ProcessOutputReader {
 
-  private static final long PROCESS_TIMEOUT_MILLIS = 10 * 60 * 1000L; // 10 minutes
-
-  private int exitCode;
+  private boolean started;
   
-  private final Process p;
+  private final Process process;
   private final ProcessStreamReader stdout;
   private final ProcessStreamReader stderr;
-  private final List<String> lines;
   
-  public ProcessOutputReader(final Process p, final ProcessStreamReader stdout, final ProcessStreamReader stderr, final List<String> lines) {
-    this.p = p;
+  public ProcessOutputReader(final Process process, final ProcessStreamReader stdout, final ProcessStreamReader stderr) {
+    this.process = process;
     this.stdout = stdout;
     this.stderr = stderr;
-    this.lines = lines;
   }
-  
-  public void waitFor() {
-    stdout.start();
-    stderr.start();
 
-    long startMillis = System.currentTimeMillis();
-    try {
-      stderr.join(PROCESS_TIMEOUT_MILLIS);
-    } catch (Exception ignore) {
+  public void start() {
+    synchronized(this) {
+      this.stdout.start();
+      this.stderr.start();
+      this.started = true;
     }
-
-    long timeLeft = System.currentTimeMillis() + PROCESS_TIMEOUT_MILLIS - startMillis;
-    try {
-      stdout.join(timeLeft);
-    } catch (Exception ignore) {
+  }
+  
+  public boolean waitFor(final long timeout, final TimeUnit unit) throws InterruptedException {
+    synchronized(this) {
+      if (!this.started) {
+        throw new IllegalStateException("Must be started before waitFor");
+      }
     }
+    
+    final long startTime = System.nanoTime();
+    
+    long millisToJoin = unit.toMillis(timeout);
+    this.stderr.join(millisToJoin);
 
-    this.exitCode = 0;
-    int retryCount = 9;
-    while (retryCount > 0) {
-      retryCount--;
+    long nanosRemaining = unit.toNanos(timeout) - (System.nanoTime() - startTime);
+    millisToJoin = unit.toMillis(nanosRemaining);
+    this.stdout.join(millisToJoin);
+
+    nanosRemaining = unit.toNanos(timeout) - (System.nanoTime() - startTime);
+    return waitForProcess(nanosRemaining, unit);
+  }
+
+  private boolean waitForProcess(final long timeout, final TimeUnit unit) throws InterruptedException {
+    long startTime = System.nanoTime();
+    long nanosRemaining = unit.toNanos(timeout);
+
+    while (nanosRemaining > 0) {
       try {
-        exitCode = p.exitValue();
-        break;
-      } catch (IllegalThreadStateException e) {
-        // due to bugs in Process we may not be able to get
-        // a process's exit value.
-        // We can't use Process.waitFor() because it can hang forever
-        if (retryCount == 0) {
-          if (stderr.linecount > 0) {
-            // The process wrote to stderr so manufacture
-            // an error exist code
-            synchronized (lines) {
-              lines.add("Failed to get exit status and it wrote"
-                  + " to stderr so setting exit status to 1.");
-            }
-            exitCode = 1;
-          }
-        } else {
-          // We need to wait around to give a chance for
-          // the child to be reaped.See bug 19682
-          try {
-            Thread.sleep(1000);
-          } catch (InterruptedException ignore) {
-          }
+        this.process.exitValue();
+        return true;
+      } catch(IllegalThreadStateException ex) {
+        if (nanosRemaining > 0) {
+          long millisToSleep =Math.min(TimeUnit.NANOSECONDS.toMillis(nanosRemaining) + 1, 100);
+          Thread.sleep(millisToSleep);
         }
       }
+      nanosRemaining = unit.toNanos(timeout) - (System.nanoTime() - startTime);
     }
-  }
-
-  public int getExitCode() {
-    return exitCode;
+    return false;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fe295940/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessStreamReader.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessStreamReader.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessStreamReader.java
index f99f5f9..de546b5 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessStreamReader.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessStreamReader.java
@@ -17,20 +17,23 @@
 package com.gemstone.gemfire.test.process;
 
 import java.io.BufferedReader;
+import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.util.List;
 import java.util.Queue;
 
 /**
- * Reads the output from a stream and stores it for test validation. Extracted
- * from ProcessWrapper.
+ * Reads the output from a process stream and stores it for test validation. 
+ * </p>
+ * Extracted from ProcessWrapper.
  * 
  * @author Kirk Lund
  */
 public class ProcessStreamReader extends Thread {
   
-  private volatile Throwable startStack;
+  private volatile RuntimeException startStack;
+  
   private final String command;
   private final BufferedReader reader;
   private final Queue<String> lineBuffer;
@@ -38,28 +41,16 @@ public class ProcessStreamReader extends Thread {
 
   public int linecount = 0;
 
-  public ProcessStreamReader(String command, InputStream stream, Queue<String> lineBuffer, List<String> allLines) {
+  public ProcessStreamReader(final String command, final InputStream stream, final Queue<String> lineBuffer, final List<String> allLines) {
     this.command = command;
     this.reader = new BufferedReader(new InputStreamReader(stream));
     this.lineBuffer = lineBuffer;
     this.allLines = allLines;
   }
 
-  public Throwable getStart() {
-    return this.startStack;
-  }
-  
-  public Throwable getFailure() {
-    if (this.startStack.getCause() != null) {
-      return this.startStack.getCause();
-    } else {
-      return null;
-    }
-  }
-  
   @Override
   public void start() {
-    this.startStack = new Throwable();
+    this.startStack = new RuntimeException(this.command);
     super.start();
   }
   
@@ -67,18 +58,17 @@ public class ProcessStreamReader extends Thread {
   public void run() {
     try {
       String line;
-      while ((line = reader.readLine()) != null) {
-        linecount++;
-        lineBuffer.offer(line);
-        allLines.add(line);
+      while ((line = this.reader.readLine()) != null) {
+        this.linecount++;
+        this.lineBuffer.offer(line);
+        this.allLines.add(line);
       }
 
       // EOF
-      reader.close();
-    } catch (Exception cause) {
-      this.startStack.initCause(cause);
-      System.err.println("ProcessStreamReader: Failure while reading from child process: " + this.command + " " + this.startStack.getMessage());
-      this.startStack.printStackTrace();
+      this.reader.close();
+    } catch (IOException streamClosed) {
+      this.startStack.initCause(streamClosed);
+      throw this.startStack;
     }
   }
 }



[04/50] [abbrv] incubator-geode git commit: Revert "GEODE-647: Adding some test debugging output"

Posted by je...@apache.org.
Revert "GEODE-647: Adding some test debugging output"

This reverts commit 9d047b4df4af8d60161cab95fc9a1af5e6d956ae.


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

Branch: refs/heads/feature/GEODE-14
Commit: b0419b1884e5c12e6998f88357e1b41de21f9856
Parents: 412f241
Author: Jens Deppe <jd...@pivotal.io>
Authored: Fri Dec 11 12:32:19 2015 -0800
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri Dec 11 13:45:01 2015 -0800

----------------------------------------------------------------------
 .../management/internal/cli/GfshParserJUnitTest.java   | 13 -------------
 1 file changed, 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b0419b18/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
index c536147..68fe251 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserJUnitTest.java
@@ -150,11 +150,8 @@ public class GfshParserJUnitTest {
    */
   @Test
   public void testComplete() throws Exception {
-    StringBuilder debugger = new StringBuilder();
-
     // get a CommandManager, add sample commands
     CommandManager commandManager = CommandManager.getInstance(false);
-    debugger.append("cm1:").append(String.format("0x%x", commandManager.hashCode()));
     assertNotNull("CommandManager should not be null.", commandManager);
     commandManager.add(Commands.class.newInstance());
     commandManager.add(SimpleConverter.class.newInstance());
@@ -170,16 +167,6 @@ public class GfshParserJUnitTest {
     List<String> completionCandidates = new ArrayList<String>();
     List<String> completionValues = new ArrayList<String>();
     parser.complete(input, input.length(), completionCandidates);
-    debugger.append(" cm2:").append(String.format("0x%x", CommandManager.getExisting().hashCode()));
-    if (completionValues.size() != completionCandidates.size()) {
-      if (completionCandidates.size() > 0) {
-        for (String s : completionCandidates) {
-          debugger.append(" ").append(s);
-        }
-      }
-      assertEquals(debugger.toString(), completionValues.size(), completionCandidates.size());
-    }
-
     assertSimpleCompletionValues(completionValues, completionCandidates);
 
     // Input contains a string which is prefix